You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/06/23 15:04:53 UTC

[GitHub] [spark] Ngone51 opened a new pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Ngone51 opened a new pull request #28911:
URL: https://github.com/apache/spark/pull/28911


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR adds support to read host-local shuffle data from disk directly when external shuffle service and dynamic allocation are both disabled.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   After SPARK-27651 / #25299, Spark can read host-local shuffle data directly from disk when external shuffle service is enabled. To extend the future, we can also support it when external shuffle service is disabled, which is useful for a static cluster where dynamic allocation is also disabled.
   
    
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   Added test and tested manually.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-686188629


   Thank you all!!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682137096


   **[Test build #127960 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127960/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655920738






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479699359



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled, 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. Note that for k8s workloads, this only works when nodes are using " +
+        "non-isolated container storage.")

Review comment:
       Good point 👍




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479691221



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.

Review comment:
       Although this is the default implementation of this abstract class, what happens when we break this assumption, `It's the same with current BlockStoreClient`? I believe we can remove this comment if this is a general reusable function for all remote host. If this is `pre-condition`, please make it sure by using `should be` instead of `is`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648226563


   **[Test build #124426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124426/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479697776



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {
+  test("read host local shuffle from disk with external shuffle service disabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, true)
+      .set(SHUFFLE_SERVICE_ENABLED, false)
+      .set(DYN_ALLOCATION_ENABLED, false)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+    sc.getConf.get(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(true)
+    sc.env.blockManager.externalShuffleServiceEnabled should equal(false)
+    sc.env.blockManager.hostLocalDirManager.isDefined should equal(true)

Review comment:
       This line adds a test coverage for `hostLocalDirManager` creation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480173840



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       hmm...I'm not sure whether @dongjoon-hyun was mentioning those two asserts. But I can try to make them as separate unit tests.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683847306


   **[Test build #128093 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128093/testReport)** for PR 28911 at commit [`6b97be5`](https://github.com/apache/spark/commit/6b97be552b1d5a78f476c4a97d794c567222d9bf).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class HostLocalShuffleReadingSuite extends SparkFunSuite with Matchers with LocalSparkContext `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693534



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +

Review comment:
       Maybe, `Error trying` -> `Error during trying` is more clear?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun closed pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #28911:
URL: https://github.com/apache/spark/pull/28911


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-659882158


   Personally, I'd save locality changes for a follow up PR. Making changes in core is pretty hard, so long as we have a JIRA and it's a good incremental chunk of work keeping it smaller for review (and potential revert if something goes wrong) is better (of course there are situations where that isn't possible, but I think changing locality calculations would be strictly additive.)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652985990


   I've updated the PR. Could you take another look?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480094857



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -120,34 +120,33 @@ private[spark] class ByteBufferBlockData(
 private[spark] class HostLocalDirManager(
     futureExecutionContext: ExecutionContext,
     cacheSize: Int,
-    externalBlockStoreClient: ExternalBlockStoreClient,
-    host: String,
-    externalShuffleServicePort: Int) extends Logging {
+    blockStoreClient: BlockStoreClient) extends Logging {
 
   private val executorIdToLocalDirsCache =
     CacheBuilder
       .newBuilder()
       .maximumSize(cacheSize)
       .build[String, Array[String]]()
 
-  private[spark] def getCachedHostLocalDirs()
-      : scala.collection.Map[String, Array[String]] = executorIdToLocalDirsCache.synchronized {
-    import scala.collection.JavaConverters._
-    return executorIdToLocalDirsCache.asMap().asScala
-  }
+  private[spark] def getCachedHostLocalDirs: Map[String, Array[String]] =
+   executorIdToLocalDirsCache.synchronized {
+      executorIdToLocalDirsCache.asMap().asScala.toMap
+   }
 
   private[spark] def getHostLocalDirs(
+      host: String,
+      port: Int,
       executorIds: Array[String])(
-      callback: Try[java.util.Map[String, Array[String]]] => Unit): Unit = {
+      callback: Try[Map[String, Array[String]]] => Unit): Unit = {

Review comment:
       It's required by `fetchMultipleHostLocalBlocks`.  Actually, we could also do the Jave to Scala map conversion before calling  `fetchMultipleHostLocalBlocks` but leaving `Try[java.util.Map[String, Array[String]]] => Unit` unchanged.
   
   But I decided to do the conversion here just because this class already imported `scala.collection.JavaConverters._`. It has no big difference to do the conversion here or there.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648227256






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661800411


   **[Test build #126234 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126234/testReport)** for PR 28911 at commit [`bcb6012`](https://github.com/apache/spark/commit/bcb60123ef68feacf4d571de47cc99068b8496b8).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `abstract class BlockTransferService extends BlockStoreClient `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456393576



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       Good point. It is true that in case of dynamic allocation executors come and go but an executor can be lost without dynamic allocation too and the code is ready to handle those errors: it leads to a `FailureFetchResult` so I expect in case of errors we are as good as before. On the other hand this still helps when local block successfully read.
   
   To sum up: I cannot see neither why dynamic allocation is excluded from this feature.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656545231


   Ah, I get your point and I can imagine how it may affect current locality preference. Let's take an example to see if we're on the same page.
   
   For example, now we have executor1 and executor2 on node1, executor3 and executor4 on node2. And we also have 10 shuffle data bytes on executor1 and executor2 from task1 and task2 separately.  Besides, we also have 40 shuffle data bytes on executor3 and executor4 from task3 and task4 separately. (Assuming all the shuffle data are for the same reduce partition.)
   
   With the current implementation of `getLocationsWithLargestOutputs`, we only count an executor's host as a locality prefer location when [shuffle data for a certain reduce partiton on this executor] / [total shuffle data]) >= fractionThreshold(default 0.2). So, in this case, only node2 is considered as a preferred location because (40 / 10 + 10 + 40 + 40) = 0.4 >= 0.2. But node1 is not because (10 / 10 + 10 + 40 + 40) = 0.1 < 0.2.
   
   However, node1 can also be a preferred location if we aggregate the size of the shuffle data on the same host, since we will have (10 + 10 / 10 + 10 + 40 + 40) = 0.2 >= 0.2.
   
   It looks reasonable to me. cc @attilapiros @tgravescs @jiangxb1987 @holdenk Any ideas?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680418219


   **[Test build #127904 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127904/testReport)** for PR 28911 at commit [`0372bd8`](https://github.com/apache/spark/commit/0372bd8a21383a021a96953ed430fc9212f8d62a).
    * This patch **fails Java style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456374577



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -194,6 +196,45 @@ private[spark] class NettyBlockTransferService(
     result.future
   }
 
+  override def getHostLocalDirs(
+      host: String,
+      port: Int,
+      execIds: Array[String],
+      hostLocalDirsCompletable: CompletableFuture[util.Map[String, Array[String]]]): Unit = {
+    val getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds)
+    try {
+      val client = clientFactory.createClient(host, port)
+      client.sendRpc(getLocalDirsMessage.toByteBuffer, new RpcResponseCallback() {
+        override def onSuccess(response: ByteBuffer): Unit = {
+          try {
+            val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response)
+            hostLocalDirsCompletable.complete(
+              msgObj.asInstanceOf[LocalDirsForExecutors].getLocalDirsByExec)
+          } catch {
+            case t: Throwable =>
+              logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+                t.getCause)
+              hostLocalDirsCompletable.completeExceptionally(t)
+          } finally {
+            client.close()
+          }
+        }
+
+        override def onFailure(t: Throwable): Unit = {
+          logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+            t.getCause)
+          hostLocalDirsCompletable.completeExceptionally(t)
+          client.close()
+        }
+      })
+    } catch {
+      case e: IOException =>

Review comment:
       Here we are catching exceptions from 
   - [TransportClientFactory#createClient()]( https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java#L149-L150)
   
   - [TransportClient#sendRpc](https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java#L188)
   
   - [TransportClient#close](https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java#L298)
   
   As I have seen only these two exceptions are relevant here. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682052038






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656656483


   **[Test build #125589 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125589/testReport)** for PR 28911 at commit [`1126341`](https://github.com/apache/spark/commit/1126341a680c3a6bea4f792516ea59b54726e11a).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648223526


   @attilapiros @tgravescs @jiangxb1987 Please take a look, thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661689855


   Jenkins, retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459759245



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       I agree with @viirya and @holdenk.
    
   @Ngone51 what would be the case when this feature is switched on (and SHUFFLE_USE_OLD_FETCH_PROTOCOL is disabled) but it cannot be used? If there is one (I cannot see it now) mentioning that the description could be simplified. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459897266



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -492,20 +494,26 @@ private[spark] class BlockManager(
       registerWithExternalShuffleServer()
     }
 
-    hostLocalDirManager =
-      if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
-        externalBlockStoreClient.map { blockStoreClient =>
-          new HostLocalDirManager(
-            futureExecutionContext,
-            conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
-            blockStoreClient,
-            blockManagerId.host,
-            externalShuffleServicePort)
-        }
+    hostLocalDirManager = {
+      val canUseHostLocalReading = conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
+        !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)
+      val externalShuffleServiceEnabled = externalBlockStoreClient.isDefined
+      val dynamicAllocationDisabled = !conf.get(config.DYN_ALLOCATION_ENABLED)
+      val dynamicAllocationEnabledWithShuffleTacking = conf.get(config.DYN_ALLOCATION_ENABLED) &&
+        conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)

Review comment:
       Thanks for catching it!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656013640


   @Ngone51 I am still catching up on changes; as part of #25299 or subsequently (or here) are we updating preferred locality for shuffle tasks to account for ability to do node local reads ?
   Essentially, all shuffle blocks on a node (irrespective of executor) should be treated with equal locality preference for computing pref locality for shuffle tasks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693064



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";

Review comment:
       If we need this, we had better move `ExternalBlockStoreClient.checkInit` to here `BlockStoreClient` in order to remove the duplication.
   ```java
     protected void checkInit() {
       assert appId != null : "Called before init()";
     }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683740444


   **[Test build #128093 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128093/testReport)** for PR 28911 at commit [`6b97be5`](https://github.com/apache/spark/commit/6b97be552b1d5a78f476c4a97d794c567222d9bf).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682048540


   Jenkins, retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663399887


   **[Test build #126487 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126487/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661676272


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661693166






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683351090


   If you want, sure! @holdenk .


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680414627


   **[Test build #127904 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127904/testReport)** for PR 28911 at commit [`0372bd8`](https://github.com/apache/spark/commit/0372bd8a21383a021a96953ed430fc9212f8d62a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661673216






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480178710



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       Thanks. I already added the checking for metrics. Let me try to add other assertions to catch the expected behavior.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684474246


   **[Test build #128139 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128139/testReport)** for PR 28911 at commit [`a23ab17`](https://github.com/apache/spark/commit/a23ab1721b1225e0a95fb27660fe81847287c622).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480847229



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -212,7 +211,7 @@ private[spark] class BlockManager(
   private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory
   private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory
 
-  private val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf)
+  private[spark] val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf)

Review comment:
       It's required by `val port = blockManager.externalShuffleServicePort` within `ShuffleBlockFetcherIterator`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648571878






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683741147






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693326



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());

Review comment:
       Indentation? IIRC, Apache Spark community use `2-space` indentation in Java.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661676275


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126227/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693534



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +

Review comment:
       Maybe, `Error trying` -> `Error during trying` or `Error while trying` is more clear?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682052038






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655943144






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480095860



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -18,15 +18,32 @@
 package org.apache.spark.network.shuffle;
 
 import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 
 import com.codahale.metrics.MetricSet;
+import org.apache.spark.network.client.RpcResponseCallback;
+import org.apache.spark.network.client.TransportClient;
+import org.apache.spark.network.client.TransportClientFactory;
+import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
+import org.apache.spark.network.shuffle.protocol.GetLocalDirsForExecutors;
+import org.apache.spark.network.shuffle.protocol.LocalDirsForExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;

Review comment:
       Thank you for catching this!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-685924904


   Retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480269400



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -212,7 +211,7 @@ private[spark] class BlockManager(
   private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory
   private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory
 
-  private val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf)
+  private[spark] val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf)

Review comment:
       Why did the scope change here?

##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
##########
@@ -113,6 +113,21 @@ class NettyBlockRpcServer(
             s"when there is not sufficient space available to store the block.")
           responseContext.onFailure(exception)
         }
+
+      case getLocalDirs: GetLocalDirsForExecutors =>
+        val isIncorrectAppId = getLocalDirs.appId != appId
+        val execNum = getLocalDirs.execIds.length
+        if (isIncorrectAppId || execNum != 1) {
+          val errorMsg = "Invalid GetLocalDirsForExecutors request: " +
+            s"${if (isIncorrectAppId) s"incorrect application id: ${getLocalDirs.appId};"}" +
+            s"${if (execNum != 1) s"incorrect executor number: $execNum (expected 1);"}"
+          responseContext.onFailure(new IllegalStateException(errorMsg))
+        } else {
+          val execId = getLocalDirs.execIds.head

Review comment:
       So this assumes that if we're passed in only one execId it's always the execId of the executor to which the block server is related? Would it make sense to do a check here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655920412


   **[Test build #125437 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125437/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652988277


   **[Test build #124905 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124905/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680418247


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/127904/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663399921


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126487/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656561347


   **[Test build #125589 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125589/testReport)** for PR 28911 at commit [`1126341`](https://github.com/apache/spark/commit/1126341a680c3a6bea4f792516ea59b54726e11a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479696374



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -463,54 +466,73 @@ final class ShuffleBlockFetcherIterator(
    * track in-memory are the ManagedBuffer references themselves.
    */
   private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = {
-    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
-    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
-      hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
-        }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
-    val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
-      }.toMap
-    if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
+    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs
+    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = {
+      val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) =>
+        cachedDirsByExec.contains(hostLocalBmId.executorId)
+      }
+      (hasCache.toMap, noCache.toMap)
+    }
+
+    if (hostLocalBlocksWithMissingDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
-        s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
-            }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+        s"${hostLocalBlocksWithMissingDirs.mkString(", ")}")
+
+      // If the external shuffle service is enabled, we'll fetch the local directories for
+      // multiple executors from the external shuffle service, which located at the same host
+      // with the executors, in once. Otherwise, we'll fetch the local directories from those
+      // executors directly one by one. The fetch requests won't be too much since one host is
+      // almost impossible to have many executors at the same time practically.
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray))
+      } else {
+        hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            fetchMultipleHostLocalBlocks(
+              hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains),
+              dirsByExecId,
+              cached = false)
+
+          case Failure(throwable) =>
+            logError("Error occurred while fetching host local blocks", throwable)
+            val bmId = bmIds.head
+            val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId)
+            val (blockId, _, mapIndex) = blockInfoSeq.head
+            results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable))
+        }
       }
     }
+
     if (hostLocalBlocksWithCachedDirs.nonEmpty) {
       logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " +
           s"${hostLocalBlocksWithCachedDirs.mkString(", ")}")
-      hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) =>
-        blockInfos.foreach { case (blockId, _, mapIndex) =>
-          if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) {
-            return
-          }
-        }
+      fetchMultipleHostLocalBlocks(hostLocalBlocksWithCachedDirs, cachedDirsByExec, cached = true)
+    }
+  }
+
+  private def fetchMultipleHostLocalBlocks(
+      bmIdToBlocks: Map[BlockManagerId, Seq[(BlockId, Long, Int)]],
+      localDirsByExecId: Map[String, Array[String]],
+      cached: Boolean)
+    : Unit = {
+    // We use `forall` because once there's a block fetch is failed, `fetchHostLocalBlock` will put

Review comment:
       Maybe, `there's a block fetch is failed,` -> `there's a failed block fetch,`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648226563


   **[Test build #124426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124426/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456393576



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       Good point. It is true that in case of dynamic allocation executors come and go but an executor can be lost without dynamic allocation too and the code is ready to handle those errors: it leads to a `FailureFetchResult` so I expect in case of errors we are as good as before. On the other hand this still helps when local block successfully read.
   
   So I cannot see the why dynamic allocation is excluded from this feature.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680423400






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r457896569



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       Yes, it prevents the case where executors could come and go in dynamic allocation. Also, I think it's still different from executor loss error. Because executor loss is an abnormal case which out of control of Spark while dynamic allocation is under control. And executor shutdown in dynamic allocation happens more frequently compares to executor loss. I think we should try our best to avoid shuffle fetch failure since its penalty is not trivial, especially when we can avoid it.
   
   Besides, for the case of dynamic allocation enabled, users could already use external shuffle service. Therefore, I can't think of a strong reason to mix these two branches.
   
   P.S. we could probably allow dynamic allocation here if `spark.dynamicAllocation.shuffleTracking.enabled` is also enabled.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656561913






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459759245



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       I agree with @viirya and @holdenk.
    
   @Ngone51 what would be the case when this feature is switched on (and SHUFFLE_USE_OLD_FETCH_PROTOCOL is disabled) but it cannot be used? If there is one maybe mentioning that the description can be simplified. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661693166






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680607978


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/127908/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656657083


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125589/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683342755


   **[Test build #128022 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128022/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r458097873



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       > As of Spark 3 we no longer require dynamic allocation to have a shuffle service.
   
   This exactly what I mentioned in P.S.. I can update the documentation according to this feature.
   
   But please also note dynamic allocation without external shuffle service is still an experimental feature disabled by default. And it has a main problem that the user needs to config when to delete shuffle files while most common users have no idea about this. And by default, shuffle files will not be removed until GC happens at the driver side. It also means executors won't come and go more frequently than dynamic allocation with shuffle service. Therefore, I think we were discussing a more general problem above when using dynamic allocation with shuffle service.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683342858






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667843527


   **[Test build #126959 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126959/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648574824


   **[Test build #124454 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124454/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662934974






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667840496






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648228080


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/124426/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684754931






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683748722


   @holdenk Sure, please feel free to add any comments!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681971335


   retest thise please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655920412


   **[Test build #125437 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125437/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693326



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());

Review comment:
       Indentation? IIRC, we use `2-space` indentation in Java and this indentation should have `4-space` instead of `8-space`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683741147






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662861471






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681646172


   Thanks for the changes @Ngone51 ! Looks good from my end.
   Will let @tgravescs (and others) do the final review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480095206



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";

Review comment:
       Sounds reasonable!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459734646



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -492,20 +494,26 @@ private[spark] class BlockManager(
       registerWithExternalShuffleServer()
     }
 
-    hostLocalDirManager =
-      if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
-        externalBlockStoreClient.map { blockStoreClient =>
-          new HostLocalDirManager(
-            futureExecutionContext,
-            conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
-            blockStoreClient,
-            blockManagerId.host,
-            externalShuffleServicePort)
-        }
+    hostLocalDirManager = {
+      val canUseHostLocalReading = conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
+        !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)
+      val externalShuffleServiceEnabled = externalBlockStoreClient.isDefined
+      val dynamicAllocationDisabled = !conf.get(config.DYN_ALLOCATION_ENABLED)
+      val dynamicAllocationEnabledWithShuffleTacking = conf.get(config.DYN_ALLOCATION_ENABLED) &&
+        conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)

Review comment:
       As said above, looks like users must be in any of the 3 conditions, i.e. (externalShuffleServiceEnabled || dynamicAllocationDisabled || dynamicAllocationEnabledWithShuffleTacking) is always be true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480162091



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       I was referring to the asserts mentioned by @dongjoon-hyun:
   - https://github.com/apache/spark/pull/28911#pullrequestreview-478159228
   - https://github.com/apache/spark/pull/28911#pullrequestreview-478159246
   
   Those could be in UTs.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480090760



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
##########
@@ -113,6 +113,15 @@ class NettyBlockRpcServer(
             s"when there is not sufficient space available to store the block.")
           responseContext.onFailure(exception)
         }
+
+      case getLocalDirs: GetLocalDirsForExecutors =>
+        assert(getLocalDirs.appId == appId)
+        assert(getLocalDirs.execIds.length == 1)

Review comment:
       This's a good point. I've changed it to reply to the sender with the error if the request fails assertion. Thus, the sender could handle the error as fetch failure.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662934445


   **[Test build #126397 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126397/testReport)** for PR 28911 at commit [`6449229`](https://github.com/apache/spark/commit/64492298c78a86920c278ab5acfda41cb4ea071f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693950



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -18,15 +18,32 @@
 package org.apache.spark.network.shuffle;
 
 import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 
 import com.codahale.metrics.MetricSet;
+import org.apache.spark.network.client.RpcResponseCallback;
+import org.apache.spark.network.client.TransportClient;
+import org.apache.spark.network.client.TransportClientFactory;
+import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
+import org.apache.spark.network.shuffle.protocol.GetLocalDirsForExecutors;
+import org.apache.spark.network.shuffle.protocol.LocalDirsForExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;

Review comment:
       This PR seems to follow the existing `import` code of `ExternalBlockStoreClient.java`. I believe new code had better be consistent to the style guide if possible.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479696711



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -463,54 +466,73 @@ final class ShuffleBlockFetcherIterator(
    * track in-memory are the ManagedBuffer references themselves.
    */
   private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = {
-    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
-    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
-      hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
-        }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
-    val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
-      }.toMap
-    if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
+    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs
+    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = {
+      val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) =>
+        cachedDirsByExec.contains(hostLocalBmId.executorId)
+      }
+      (hasCache.toMap, noCache.toMap)
+    }
+
+    if (hostLocalBlocksWithMissingDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
-        s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
-            }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+        s"${hostLocalBlocksWithMissingDirs.mkString(", ")}")
+
+      // If the external shuffle service is enabled, we'll fetch the local directories for
+      // multiple executors from the external shuffle service, which located at the same host
+      // with the executors, in once. Otherwise, we'll fetch the local directories from those
+      // executors directly one by one. The fetch requests won't be too much since one host is
+      // almost impossible to have many executors at the same time practically.
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray))
+      } else {
+        hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            fetchMultipleHostLocalBlocks(
+              hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains),
+              dirsByExecId,
+              cached = false)
+
+          case Failure(throwable) =>
+            logError("Error occurred while fetching host local blocks", throwable)
+            val bmId = bmIds.head
+            val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId)
+            val (blockId, _, mapIndex) = blockInfoSeq.head
+            results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable))
+        }
       }
     }
+
     if (hostLocalBlocksWithCachedDirs.nonEmpty) {
       logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " +
           s"${hostLocalBlocksWithCachedDirs.mkString(", ")}")
-      hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) =>
-        blockInfos.foreach { case (blockId, _, mapIndex) =>
-          if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) {
-            return
-          }
-        }
+      fetchMultipleHostLocalBlocks(hostLocalBlocksWithCachedDirs, cachedDirsByExec, cached = true)
+    }
+  }
+
+  private def fetchMultipleHostLocalBlocks(
+      bmIdToBlocks: Map[BlockManagerId, Seq[(BlockId, Long, Int)]],
+      localDirsByExecId: Map[String, Array[String]],
+      cached: Boolean)
+    : Unit = {
+    // We use `forall` because once there's a block fetch is failed, `fetchHostLocalBlock` will put
+    // a `FailureFetchResult` immediately to the `results`. So there's no reason to fetch the
+    // remaining blocks.
+    val allFetchSucceed = bmIdToBlocks.forall { case (bmId, blockInfos) =>

Review comment:
       `allFetchSucceed` -> `allFetchSucceeded` because we usually use `XXXSucceeded`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479695389



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
##########
@@ -113,6 +113,15 @@ class NettyBlockRpcServer(
             s"when there is not sufficient space available to store the block.")
           responseContext.onFailure(exception)
         }
+
+      case getLocalDirs: GetLocalDirsForExecutors =>
+        assert(getLocalDirs.appId == appId)
+        assert(getLocalDirs.execIds.length == 1)

Review comment:
       Although we have these assertions, what happen when this `NettyBlockRpcServer` receives a wrong message, `GetLocalDirsForExecutors` with an irrelevant `execId`? It seems that we trust the sender always and reply with the received `execId`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680415445


   Thank you @tgravescs . I've rebased the PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681543539






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459673702



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -466,42 +464,51 @@ final class ShuffleBlockFetcherIterator(
     val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
     val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
       hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
+        .map { case (hostLocalBmId, blockInfos) =>
+          (hostLocalBmId, blockInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
         }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
     val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
+      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, blockInfos, _) =>
+        hostLocalBmId -> blockInfos
       }.toMap
     if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        immutableHostLocalBlocksWithoutDirs.keys
+          .map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            bmIds.foreach { bmId =>
+              val dirs = dirsByExecId.get(bmId.executorId)
+              immutableHostLocalBlocksWithoutDirs(bmId)
+                .takeWhile { case (blockId, _, mapIndex) =>
+                  fetchHostLocalBlock(blockId, mapIndex, dirs, bmId)
+                }
             }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+            logDebug(s"Got host-local blocks (without cached executors' dir) in " +
+              s"${Utils.getUsedTimeNs(startTimeNs)}")
+
+          case Failure(throwable) =>
+            logError(s"Error occurred while fetching host local blocks", throwable)

Review comment:
       s"" is not needed.

##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -466,42 +464,51 @@ final class ShuffleBlockFetcherIterator(
     val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
     val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
       hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
+        .map { case (hostLocalBmId, blockInfos) =>
+          (hostLocalBmId, blockInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
         }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
     val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
+      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, blockInfos, _) =>
+        hostLocalBmId -> blockInfos
       }.toMap
     if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        immutableHostLocalBlocksWithoutDirs.keys
+          .map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            bmIds.foreach { bmId =>
+              val dirs = dirsByExecId.get(bmId.executorId)
+              immutableHostLocalBlocksWithoutDirs(bmId)
+                .takeWhile { case (blockId, _, mapIndex) =>
+                  fetchHostLocalBlock(blockId, mapIndex, dirs, bmId)
+                }
             }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+            logDebug(s"Got host-local blocks (without cached executors' dir) in " +

Review comment:
       first s"" is not needed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683748443


   Thank you @dongjoon-hyun for the detailed review. It helps a lot to improve PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656561913






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989659






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667843605


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126959/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680414930






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648571878






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656234880


   @tgravescs Thanks for the review. I'll try to address them tomorrow.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480847421



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
##########
@@ -113,6 +113,21 @@ class NettyBlockRpcServer(
             s"when there is not sufficient space available to store the block.")
           responseContext.onFailure(exception)
         }
+
+      case getLocalDirs: GetLocalDirsForExecutors =>
+        val isIncorrectAppId = getLocalDirs.appId != appId
+        val execNum = getLocalDirs.execIds.length
+        if (isIncorrectAppId || execNum != 1) {
+          val errorMsg = "Invalid GetLocalDirsForExecutors request: " +
+            s"${if (isIncorrectAppId) s"incorrect application id: ${getLocalDirs.appId};"}" +
+            s"${if (execNum != 1) s"incorrect executor number: $execNum (expected 1);"}"
+          responseContext.onFailure(new IllegalStateException(errorMsg))
+        } else {
+          val execId = getLocalDirs.execIds.head

Review comment:
       Make sense to me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667839615


   **[Test build #126959 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126959/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667837875


   @tgravescs @mridulm @holdenk @viirya @attilapiros Any more concerns?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648571518


   **[Test build #124454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124454/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648228067


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r457876162



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +63,17 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories, which are specified by DiskBlockManager, for the executors
+   * from the external shuffle service (when this is a ExternalBlockStoreClient) or BlockManager
+   * (when this is a NettyBlockTransferService). Note there's only one executor when this is a
+   * NettyBlockTransferService because we ask one specific executor at a time.

Review comment:
       I added the check to ensure it's the only one executor id but didn't check its equality with blockManager's executor id. Because we only have `BlockDataManager` in `NettyBlockRpcServer` which does not expose executor id. 
   
   I am still wondering whether it's worthwhile to expose it for the sanity check purpose.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456393417



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +63,17 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories, which are specified by DiskBlockManager, for the executors
+   * from the external shuffle service (when this is a ExternalBlockStoreClient) or BlockManager
+   * (when this is a NettyBlockTransferService). Note there's only one executor when this is a
+   * NettyBlockTransferService because we ask one specific executor at a time.

Review comment:
       Good point. It is true that in case of dynamic allocation executors come and go but an executor can be lost without dynamic allocation too and the code is prepared to handle those: it leads to a `FailureFetchResult` so I expect in case of errors we are as good as before. On the other hand this still helps when local block successfully read.
   
   So I cannot see the why dynamic allocation is excluded from this feature.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693326



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());

Review comment:
       Indentation? IIRC, Apache Spark community use `2-space` indentation in Java and this line's indentation should have `4-space` instead of `8-space`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479698150



##########
File path: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
##########
@@ -66,16 +66,24 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
     transfer
   }
 
+  private def createMockBlockManager(): BlockManager = {
+    val blockManager = mock(classOf[BlockManager])
+    val localBmId = BlockManagerId("test-client", "test-local-host", 1)
+    doReturn(localBmId).when(blockManager).blockManagerId
+    // By default, the mock BlockManager returns None for hostLocalDirManager. One could
+    // still use initHostLocalDirManager() to specify a custom hostLocalDirManager.

Review comment:
       Just a question. Is there an instance which is not using this default?

##########
File path: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
##########
@@ -66,16 +66,24 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
     transfer
   }
 
+  private def createMockBlockManager(): BlockManager = {
+    val blockManager = mock(classOf[BlockManager])
+    val localBmId = BlockManagerId("test-client", "test-local-host", 1)
+    doReturn(localBmId).when(blockManager).blockManagerId
+    // By default, the mock BlockManager returns None for hostLocalDirManager. One could
+    // still use initHostLocalDirManager() to specify a custom hostLocalDirManager.

Review comment:
       Just a question. Currently, is there an instance which is not using this default?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r452705081



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -194,6 +196,45 @@ private[spark] class NettyBlockTransferService(
     result.future
   }
 
+  override def getHostLocalDirs(
+      host: String,
+      port: Int,
+      execIds: Array[String],
+      hostLocalDirsCompletable: CompletableFuture[util.Map[String, Array[String]]]): Unit = {
+    val getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds)
+    try {
+      val client = clientFactory.createClient(host, port)
+      client.sendRpc(getLocalDirsMessage.toByteBuffer, new RpcResponseCallback() {
+        override def onSuccess(response: ByteBuffer): Unit = {
+          try {
+            val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response)
+            hostLocalDirsCompletable.complete(
+              msgObj.asInstanceOf[LocalDirsForExecutors].getLocalDirsByExec)
+          } catch {
+            case t: Throwable =>
+              logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+                t.getCause)
+              hostLocalDirsCompletable.completeExceptionally(t)
+          } finally {
+            client.close()
+          }
+        }
+
+        override def onFailure(t: Throwable): Unit = {
+          logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+            t.getCause)
+          hostLocalDirsCompletable.completeExceptionally(t)
+          client.close()
+        }
+      })
+    } catch {
+      case e: IOException =>

Review comment:
       This is migrated from `ExternalBlockStoreClient.getHostLocalDirs`. Maybe, @attilapiros has more context?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459759245



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       I agree with @viirya and @holdenk.
    
   @Ngone51 what would be the case when this feature is switched on (and SHUFFLE_USE_OLD_FETCH_PROTOCOL is disabled) but it cannot be used? If there is maybe with that the description can be simplified. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683740444


   **[Test build #128093 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128093/testReport)** for PR 28911 at commit [`6b97be5`](https://github.com/apache/spark/commit/6b97be552b1d5a78f476c4a97d794c567222d9bf).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662509102


   ping @squito Could you also take a look? Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661673216






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680607948


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r478339051



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -463,54 +466,73 @@ final class ShuffleBlockFetcherIterator(
    * track in-memory are the ManagedBuffer references themselves.
    */
   private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = {
-    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
-    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
-      hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
-        }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
-    val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
-      }.toMap
-    if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
+    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs
+    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = {
+      val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) =>
+        cachedDirsByExec.contains(hostLocalBmId.executorId)
+      }
+      (hasCache.toMap, noCache.toMap)
+    }
+
+    if (hostLocalBlocksWithMissingDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
-        s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
-            }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+        s"${hostLocalBlocksWithMissingDirs.mkString(", ")}")
+
+      // If the external shuffle service is enabled, we'll fetch the local directories for
+      // multiple executors from the external shuffle service, which located at the same host
+      // with the executors, in once. Otherwise, we'll fetch the local directories from those
+      // executors directly one by one. The fetch requests won't be too much since one host is
+      // almost impossible to have many executors at the same time practically.
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray))
+      } else {
+        hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            fetchMultipleHostLocalBlocks(
+              hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains),
+              dirsByExecId,
+              cached = false)
+
+          case Failure(throwable) =>
+            logError("Error occurred while fetching host local blocks", throwable)
+            val bmId = bmIds.head
+            val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId)
+            val (blockId, _, mapIndex) = blockInfoSeq.head
+            results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable))
+        }
       }
     }
+
     if (hostLocalBlocksWithCachedDirs.nonEmpty) {
       logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " +
           s"${hostLocalBlocksWithCachedDirs.mkString(", ")}")
-      hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) =>
-        blockInfos.foreach { case (blockId, _, mapIndex) =>
-          if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) {

Review comment:
       @Ngone51 Nice catch! Thanks for fixing it in `fetchMultipleHostLocalBlocks`. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656657077






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681619106


   Thank you for the review. I've addressed the comments. Please take another look when you have time.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684751065


   **[Test build #128139 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128139/testReport)** for PR 28911 at commit [`a23ab17`](https://github.com/apache/spark/commit/a23ab1721b1225e0a95fb27660fe81847287c622).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680607948






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680423095


   **[Test build #127908 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127908/testReport)** for PR 28911 at commit [`17f1b60`](https://github.com/apache/spark/commit/17f1b60a49da1d64e8df03400ba96bed750440ae).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682051359


   **[Test build #127960 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127960/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479698023



##########
File path: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
##########
@@ -66,16 +66,24 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
     transfer
   }
 
+  private def createMockBlockManager(): BlockManager = {

Review comment:
       Thank you for adding this helper.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479696967



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       Thank you for adding this, but this adds only two minor test coverage which can be replaced by smaller UT. This test suite looks like an IT or a test runner for debugging.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663397663






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680423400






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655943157


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125437/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459264839



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       Please take a look at the updated document regarding the dynamic allocation. cc: @holdenk @attilapiros 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989659


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683848867






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682137992






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684474246


   **[Test build #128139 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128139/testReport)** for PR 28911 at commit [`a23ab17`](https://github.com/apache/spark/commit/a23ab1721b1225e0a95fb27660fe81847287c622).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989009






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661672706


   **[Test build #126227 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126227/testReport)** for PR 28911 at commit [`bcb6012`](https://github.com/apache/spark/commit/bcb60123ef68feacf4d571de47cc99068b8496b8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662934974






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656234537


   @mridulm we don't and no need to do it. The current implementation of `getPreferredLocationsForShuffle` already treats blocks on the same node as the same locality preference(see L617):
   
   https://github.com/apache/spark/blob/6fcb70e0cadd8a543cd9be5f606c5dbeec0ae181/core/src/main/scala/org/apache/spark/MapOutputTracker.scala#L610-L624
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683355864


   **[Test build #128022 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128022/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480171224



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled, 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. Note that for k8s workloads, this only works when nodes are using " +
+        "non-isolated container storage.")

Review comment:
       Sorry for the confusion. 
   
   My statement: this feature does not work on containerized systems even on non-isolated storage based containers as on those system AFAIK all the containers (running the executors and block managers along) have different hostnames (and IP addresses).
   
   This feature currently is built on detecting the shared storage by the reused hostnames:
   https://github.com/apache/spark/blob/6b97be552b1d5a78f476c4a97d794c567222d9bf/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala#L307
   
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479691221



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.

Review comment:
       Although this is the default implementation of this abstract class, what happens when we break this assumption, `It's the same with current BlockStoreClient`? I believe we can remove this comment if this is a general reusable function for all remote host.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681654515


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667843579


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r457876717



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -194,6 +196,45 @@ private[spark] class NettyBlockTransferService(
     result.future
   }
 
+  override def getHostLocalDirs(
+      host: String,
+      port: Int,
+      execIds: Array[String],
+      hostLocalDirsCompletable: CompletableFuture[util.Map[String, Array[String]]]): Unit = {
+    val getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds)
+    try {
+      val client = clientFactory.createClient(host, port)
+      client.sendRpc(getLocalDirsMessage.toByteBuffer, new RpcResponseCallback() {
+        override def onSuccess(response: ByteBuffer): Unit = {
+          try {
+            val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response)
+            hostLocalDirsCompletable.complete(
+              msgObj.asInstanceOf[LocalDirsForExecutors].getLocalDirsByExec)
+          } catch {
+            case t: Throwable =>
+              logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+                t.getCause)
+              hostLocalDirsCompletable.completeExceptionally(t)
+          } finally {
+            client.close()
+          }
+        }
+
+        override def onFailure(t: Throwable): Unit = {
+          logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+            t.getCause)
+          hostLocalDirsCompletable.completeExceptionally(t)
+          client.close()
+        }
+      })
+    } catch {
+      case e: IOException =>

Review comment:
       Yes. Good idea. I've did the refactor. Please take a look.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680414930






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648227256






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479695043



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -492,20 +493,17 @@ private[spark] class BlockManager(
       registerWithExternalShuffleServer()
     }
 
-    hostLocalDirManager =
+    hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
-        externalBlockStoreClient.map { blockStoreClient =>
-          new HostLocalDirManager(
-            futureExecutionContext,
-            conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
-            blockStoreClient,
-            blockManagerId.host,
-            externalShuffleServicePort)
-        }
+        !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {

Review comment:
       indentation.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -492,20 +493,17 @@ private[spark] class BlockManager(
       registerWithExternalShuffleServer()
     }
 
-    hostLocalDirManager =
+    hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
-        externalBlockStoreClient.map { blockStoreClient =>
-          new HostLocalDirManager(
-            futureExecutionContext,
-            conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
-            blockStoreClient,
-            blockManagerId.host,
-            externalShuffleServicePort)
-        }
+        !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {

Review comment:
       indentation?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 closed pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 closed pull request #28911:
URL: https://github.com/apache/spark/pull/28911


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459902032



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,62 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id to its
+   *                                 local directories if the request handler replies successfully.
+   *                                 Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+                            Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service",

Review comment:
       Thanks for catching this. I just removed "via external shuffle service" since the `logger` would print the current class name. Do you think it's ok?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684476666






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459671216



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,62 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id to its
+   *                                 local directories if the request handler replies successfully.
+   *                                 Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+                            Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service",

Review comment:
       "via external shuffle service" -> "via BlockStoreClient"?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,62 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id to its
+   *                                 local directories if the request handler replies successfully.
+   *                                 Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+                            Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service",
+                    t.getCause());
+            hostLocalDirsCompletable.completeExceptionally(t);
+          } finally {
+            client.close();
+          }
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          logger.warn("Error trying to get the host local dirs for " +
+            Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service",

Review comment:
       ditto.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648574961






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479691221



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.

Review comment:
       What happens when we break this assumption, `It's the same with current BlockStoreClient`? I believe we can remove this comment if this is a general reusable function for all remote host.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456399850



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +63,17 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories, which are specified by DiskBlockManager, for the executors
+   * from the external shuffle service (when this is a ExternalBlockStoreClient) or BlockManager
+   * (when this is a NettyBlockTransferService). Note there's only one executor when this is a
+   * NettyBlockTransferService because we ask one specific executor at a time.

Review comment:
       Oh I got this. 
   
   When the external shuffle service is target by this request we can collect the local dirs of multiple executors at once (as all the host local dirs are available in the external shuffle service running on the host as it is central component in this sense on that host).
   
   But here we can request the local dirs for only one executor the one which handles the request itself.
   
   @Ngone51 what about adding an assert here: https://github.com/apache/spark/blob/1126341a680c3a6bea4f792516ea59b54726e11a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala#L119
   Checking the array only contains one executor ID and it is equal with the `executorId` of the `blockManager`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680040033


   @Ngone51 sorry, I missed your last update, can you up merge and I'll take another look at it


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683749846


   @attilapiros Updated, thanks for the reminder!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683356005






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681977619


   retest this please.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480176296



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       I think you can keep those as it is just extend this integration test with more assert to make sure the feature really works (through all the components). This is why I suggested to check the metrics too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480157384



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled, 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. Note that for k8s workloads, this only works when nodes are using " +
+        "non-isolated container storage.")

Review comment:
       > Currently this is done by using the some host in the blockmanager ID which works only for YARN and standalone mode, is not it?
   
   IIUC, from @holdenk 's previous comment and @dongjoon-hyun 's comment, it should also work for Mesos/K8s when they're using the non-isolated container.
   
   
   > A question for the future: do you have a plan to introduce block manager grouping based on shared storage?
   
   I don't. To be honest, I'm not familiar with the containerized resource manager. I'm also not sure what the plan you're meaning here. Is it only needed for the containerized resource manager?

##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       Sorry, which two asserts are you referring to? Are these two asserts in the new test:
   
   ```scala
   // Spark should read the shuffle data locally from the cached directories on the same host,
   // so there's no remote fetching at all.
   assert(localBytesRead.sum > 0)
   assert(remoteBytesRead.sum === 0)
   ```
   If they are, I actually think that checking `localBytesRead`/`remoteBytesRead` is equal to `localBlocksFetched`/`remoteBlocksFetched` here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648228052


   **[Test build #124426 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124426/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).
    * This patch **fails build dependency tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683848867






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989662


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/124905/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r458088179



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       As of Spark 3 we no longer require dynamic allocation to have a shuffle service. Can you explain what would fail in the current approach with dynamic allocation which requires disabiling it? And also document that it does not function with dynamic allocation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656595748


   The fix for this need not necessarily come in this PR, but can be a feature addition.
   Note that local shuffle reads from across executors on a node will really benefit when locality preference also accounts for it - until then, the potential benefits will be reduced.
   
   The solution is fairly straightforward, given existing implementation of `getLocationsWithLargestOutputs` - when aggregating, aggregate by host instead of blockmanager id when local reads across executors on a node are possible. This PR, #25299 are candidates when this can be enabled (with suitable flag checks, etc).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683342139


   Retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656401605


   The output is host, but size computation does not aggregate by host.
   
   
   On Thu, Jul 9, 2020 at 9:45 AM wuyi <no...@github.com> wrote:
   
   > @tgravescs <https://github.com/tgravescs> Thanks for the review. I'll try
   > to address them tomorrow.
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/spark/pull/28911#issuecomment-656234880>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AAMETFBZGIMLWOFUE234CBDR2XX4NANCNFSM4OFYIFCQ>
   > .
   >
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655942846


   **[Test build #125437 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125437/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r447553884



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.Matchers
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {
+  test("read host local shuffle from disk with external shuffle service disabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, true)
+      .set(SHUFFLE_SERVICE_ENABLED, false)
+      .set(DYN_ALLOCATION_ENABLED, false)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+    sc.getConf.get(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(true)
+    sc.env.blockManager.externalShuffleServiceEnabled should equal(false)
+    sc.env.blockManager.hostLocalDirManager.isDefined should equal(true)
+    sc.env.blockManager.blockStoreClient.getClass should equal(classOf[NettyBlockTransferService])
+    TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
+
+    val rdd = sc.parallelize(0 until 1000, 10)
+      .map { i => (i, 1) }
+      .reduceByKey(_ + _)
+
+    rdd.count()
+    rdd.count()
+
+    val cachedExecutors = rdd.mapPartitions { _ =>
+      SparkEnv.get.blockManager.hostLocalDirManager.map { localDirManager =>
+        localDirManager.getCachedHostLocalDirs().keySet.iterator
+      }.getOrElse(Iterator.empty)
+    }.collect().toSet
+
+    // both executors are caching the dirs of the other one
+    cachedExecutors should equal(sc.getExecutorIds().toSet)
+
+    // Now Spark will not receive FetchFailed as host local blocks are read from the cached local
+    // disk directly

Review comment:
       I see, thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662861471






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479694798



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -120,34 +120,33 @@ private[spark] class ByteBufferBlockData(
 private[spark] class HostLocalDirManager(
     futureExecutionContext: ExecutionContext,
     cacheSize: Int,
-    externalBlockStoreClient: ExternalBlockStoreClient,
-    host: String,
-    externalShuffleServicePort: Int) extends Logging {
+    blockStoreClient: BlockStoreClient) extends Logging {
 
   private val executorIdToLocalDirsCache =
     CacheBuilder
       .newBuilder()
       .maximumSize(cacheSize)
       .build[String, Array[String]]()
 
-  private[spark] def getCachedHostLocalDirs()
-      : scala.collection.Map[String, Array[String]] = executorIdToLocalDirsCache.synchronized {
-    import scala.collection.JavaConverters._
-    return executorIdToLocalDirsCache.asMap().asScala
-  }
+  private[spark] def getCachedHostLocalDirs: Map[String, Array[String]] =
+   executorIdToLocalDirsCache.synchronized {
+      executorIdToLocalDirsCache.asMap().asScala.toMap
+   }

Review comment:
       indentation for line 132 ~ 134?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r458099668



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       I still don’t see what would not work with this feature, maybe you can try and explain it in another manner?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989644


   **[Test build #124905 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124905/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).
    * This patch **fails build dependency tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684754931






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667840496






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662863820


   **[Test build #126397 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126397/testReport)** for PR 28911 at commit [`6449229`](https://github.com/apache/spark/commit/64492298c78a86920c278ab5acfda41cb4ea071f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667839615


   **[Test build #126959 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126959/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479690817



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -18,15 +18,32 @@
 package org.apache.spark.network.shuffle;
 
 import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 
 import com.codahale.metrics.MetricSet;
+import org.apache.spark.network.client.RpcResponseCallback;
+import org.apache.spark.network.client.TransportClient;
+import org.apache.spark.network.client.TransportClientFactory;
+import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
+import org.apache.spark.network.shuffle.protocol.GetLocalDirsForExecutors;
+import org.apache.spark.network.shuffle.protocol.LocalDirsForExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;

Review comment:
       Hi, @Ngone51 . Could you follow the community guide for `import` order, please?
   - https://github.com/databricks/scala-style-guide/blame/master/README.md#L403-L408




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648569632


   Jenkins, retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648574961


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680423095


   **[Test build #127908 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127908/testReport)** for PR 28911 at commit [`17f1b60`](https://github.com/apache/spark/commit/17f1b60a49da1d64e8df03400ba96bed750440ae).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682394140


   @tgravescs @mridulm thanks for the approval. It's ready to merge.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681653076


   **[Test build #127945 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127945/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682051359


   **[Test build #127960 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127960/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681543539






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693064



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";

Review comment:
       If we need this assertion, we had better move `ExternalBlockStoreClient.checkInit` function to here, `BlockStoreClient`, in order to remove the duplication.
   ```java
     protected void checkInit() {
       assert appId != null : "Called before init()";
     }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456393576



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       Good point. It is true that in case of dynamic allocation executors come and go but an executor can be lost without dynamic allocation too and the code is prepared to handle those: it leads to a FailureFetchResult so I expect in case of errors we are as good as before. On the other hand this still helps when local block successfully read.
   
   So I cannot see the why dynamic allocation is excluded from this feature.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r452703876



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -478,30 +475,43 @@ final class ShuffleBlockFetcherIterator(
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
       val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        hostLocalBlocksByExecutor.keysIterator

Review comment:
       yea, correct! Updated to resue the `immutableHostLocalBlocksWithoutDirs`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r477013033



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,60 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                to its local directories if the request handler replies
+   *                                successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+              Arrays.toString(getLocalDirsMessage.execIds), t.getCause());
+            hostLocalDirsCompletable.completeExceptionally(t);
+          } finally {
+            client.close();
+          }
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          logger.warn("Error trying to get the host local dirs for " +
+            Arrays.toString(getLocalDirsMessage.execIds), t.getCause());
+          hostLocalDirsCompletable.completeExceptionally(t);
+          client.close();

Review comment:
       Same as above, dont close `client`.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,60 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                to its local directories if the request handler replies
+   *                                successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+              Arrays.toString(getLocalDirsMessage.execIds), t.getCause());
+            hostLocalDirsCompletable.completeExceptionally(t);
+          } finally {
+            client.close();

Review comment:
       `client` is cached - they should not be closed.
   Other in-flight requests will fail.

##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -466,42 +464,51 @@ final class ShuffleBlockFetcherIterator(
     val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
     val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
       hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
+        .map { case (hostLocalBmId, blockInfos) =>
+          (hostLocalBmId, blockInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
         }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
     val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
+      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, blockInfos, _) =>
+        hostLocalBmId -> blockInfos
       }.toMap
     if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        immutableHostLocalBlocksWithoutDirs.keys
+          .map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>

Review comment:
       Shouldn't this be exactly same as `hostLocalBlocksWithCachedDirs.isDefined` block below ?
   If yes, we should refactor them out

##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -113,7 +113,7 @@ private[spark] class NettyBlockTransferService(
       blockIds: Array[String],
       listener: BlockFetchingListener,
       tempFileManager: DownloadFileManager): Unit = {
-    logTrace(s"Fetch blocks from $host:$port (executor id $execId)")
+    logger.trace(s"Fetch blocks from $host:$port (executor id $execId)")

Review comment:
       Wrap it in `if (logger.isTraceEnabled)` to prevent unnecessary string interpolation.
   Please do this for other trace/debug in this file




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 closed pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 closed pull request #28911:
URL: https://github.com/apache/spark/pull/28911


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680605972


   **[Test build #127908 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127908/testReport)** for PR 28911 at commit [`17f1b60`](https://github.com/apache/spark/commit/17f1b60a49da1d64e8df03400ba96bed750440ae).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r478175273



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -466,42 +464,51 @@ final class ShuffleBlockFetcherIterator(
     val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
     val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
       hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
+        .map { case (hostLocalBmId, blockInfos) =>
+          (hostLocalBmId, blockInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
         }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
     val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
+      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, blockInfos, _) =>
+        hostLocalBmId -> blockInfos
       }.toMap
     if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        immutableHostLocalBlocksWithoutDirs.keys
+          .map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>

Review comment:
       yes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r447554793



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +
+        s"the same host are read from the disk directly instead of being fetched as remote blocks" +
+        s" over the network.")

Review comment:
       Do you mean, in k8s, this feature should only work when executors are using non-isolated container storage?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456399850



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +63,17 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories, which are specified by DiskBlockManager, for the executors
+   * from the external shuffle service (when this is a ExternalBlockStoreClient) or BlockManager
+   * (when this is a NettyBlockTransferService). Note there's only one executor when this is a
+   * NettyBlockTransferService because we ask one specific executor at a time.

Review comment:
       Oh I got this. 
   
   When the external shuffle service is target by this request we can collect the local dirs of multiple executors at once (as all the host local dirs are available in the external shuffle service running on the host as it is central component in this sense on that host).
   
   But here we can request the local dirs for only one executor: the one which handles the request itself.
   
   @Ngone51 what about adding an assert here: https://github.com/apache/spark/blob/1126341a680c3a6bea4f792516ea59b54726e11a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala#L119
   Checking the array only contains one executor ID and it is equal with the `executorId` of the `blockManager`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655943144


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480086501



##########
File path: core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
##########
@@ -66,16 +66,24 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
     transfer
   }
 
+  private def createMockBlockManager(): BlockManager = {
+    val blockManager = mock(classOf[BlockManager])
+    val localBmId = BlockManagerId("test-client", "test-local-host", 1)
+    doReturn(localBmId).when(blockManager).blockManagerId
+    // By default, the mock BlockManager returns None for hostLocalDirManager. One could
+    // still use initHostLocalDirManager() to specify a custom hostLocalDirManager.

Review comment:
       tests like:
   
   `successful 3 local + 4 host local + 2 remote reads`
   `fetch continuous blocks in batch successful 3 local + 4 host local + 2 remote reads`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652988277


   **[Test build #124905 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124905/testReport)** for PR 28911 at commit [`da14484`](https://github.com/apache/spark/commit/da14484bb520662372b739078d4aa78433ac46a3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r478179838



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -463,54 +466,73 @@ final class ShuffleBlockFetcherIterator(
    * track in-memory are the ManagedBuffer references themselves.
    */
   private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = {
-    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs()
-    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) =
-      hostLocalBlocksByExecutor
-        .map { case (hostLocalBmId, bmInfos) =>
-          (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId))
-        }.partition(_._3.isDefined)
-    val bmId = blockManager.blockManagerId
-    val immutableHostLocalBlocksWithoutDirs =
-      hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) =>
-        hostLocalBmId -> bmInfos
-      }.toMap
-    if (immutableHostLocalBlocksWithoutDirs.nonEmpty) {
+    val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs
+    val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = {
+      val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) =>
+        cachedDirsByExec.contains(hostLocalBmId.executorId)
+      }
+      (hasCache.toMap, noCache.toMap)
+    }
+
+    if (hostLocalBlocksWithMissingDirs.nonEmpty) {
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
-        s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
-      val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
-            }
-          }
-          logDebug(s"Got host-local blocks (without cached executors' dir) in " +
-            s"${Utils.getUsedTimeNs(startTimeNs)}")
-
-        case Failure(throwable) =>
-          logError(s"Error occurred while fetching host local blocks", throwable)
-          val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head
-          val (blockId, _, mapIndex) = blockInfoSeq.head
-          results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable))
+        s"${hostLocalBlocksWithMissingDirs.mkString(", ")}")
+
+      // If the external shuffle service is enabled, we'll fetch the local directories for
+      // multiple executors from the external shuffle service, which located at the same host
+      // with the executors, in once. Otherwise, we'll fetch the local directories from those
+      // executors directly one by one. The fetch requests won't be too much since one host is
+      // almost impossible to have many executors at the same time practically.
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray))
+      } else {
+        hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq
+      }
+
+      dirFetchRequests.foreach { case (host, port, bmIds) =>
+        hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
+          case Success(dirsByExecId) =>
+            fetchMultipleHostLocalBlocks(
+              hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains),
+              dirsByExecId,
+              cached = false)
+
+          case Failure(throwable) =>
+            logError("Error occurred while fetching host local blocks", throwable)
+            val bmId = bmIds.head
+            val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId)
+            val (blockId, _, mapIndex) = blockInfoSeq.head
+            results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable))
+        }
       }
     }
+
     if (hostLocalBlocksWithCachedDirs.nonEmpty) {
       logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " +
           s"${hostLocalBlocksWithCachedDirs.mkString(", ")}")
-      hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) =>
-        blockInfos.foreach { case (blockId, _, mapIndex) =>
-          if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) {

Review comment:
       @attilapiros  This looks like a bug before. The `bmId` is for the current executor but blocks can be other executors on the same node. So, those blocks are not included as local read bytes at the end:
   https://github.com/apache/spark/blob/eb379766f406fc1f91821f9109bacff7f3403fc3/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala#L575-L580
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r448971785



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +
+        s"the same host are read from the disk directly instead of being fetched as remote blocks" +
+        s" over the network.")

Review comment:
       I'm not familiar with cloud env. So I ask the question to verify my understanding.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680414627


   **[Test build #127904 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127904/testReport)** for PR 28911 at commit [`0372bd8`](https://github.com/apache/spark/commit/0372bd8a21383a021a96953ed430fc9212f8d62a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663399910


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648228067






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480121001



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       I see value in this test but @dongjoon-hyun is right too (currently the two asserts can be done with some small unit tests too). 
   
   To improve this I would make sure that really our feature worked under the hood. 
   For example I would check the `ShuffleReadMetrics` as there should be no remote block fetching.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r452276119



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -478,30 +475,43 @@ final class ShuffleBlockFetcherIterator(
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
       val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        hostLocalBlocksByExecutor.keysIterator

Review comment:
       isn't this just essentially calculating immutableHostLocalBlocksWithoutDirs again? That contains the bmids of the ones missing from the cache.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +63,17 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories, which are specified by DiskBlockManager, for the executors
+   * from the external shuffle service (when this is a ExternalBlockStoreClient) or BlockManager
+   * (when this is a NettyBlockTransferService). Note there's only one executor when this is a
+   * NettyBlockTransferService.

Review comment:
       might be nice to extend saying because we ask one specific executor at a time.

##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -194,6 +196,45 @@ private[spark] class NettyBlockTransferService(
     result.future
   }
 
+  override def getHostLocalDirs(
+      host: String,
+      port: Int,
+      execIds: Array[String],
+      hostLocalDirsCompletable: CompletableFuture[util.Map[String, Array[String]]]): Unit = {
+    val getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds)
+    try {
+      val client = clientFactory.createClient(host, port)
+      client.sendRpc(getLocalDirsMessage.toByteBuffer, new RpcResponseCallback() {
+        override def onSuccess(response: ByteBuffer): Unit = {
+          try {
+            val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response)
+            hostLocalDirsCompletable.complete(
+              msgObj.asInstanceOf[LocalDirsForExecutors].getLocalDirsByExec)
+          } catch {
+            case t: Throwable =>
+              logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+                t.getCause)
+              hostLocalDirsCompletable.completeExceptionally(t)
+          } finally {
+            client.close()
+          }
+        }
+
+        override def onFailure(t: Throwable): Unit = {
+          logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+            t.getCause)
+          hostLocalDirsCompletable.completeExceptionally(t)
+          client.close()
+        }
+      })
+    } catch {
+      case e: IOException =>

Review comment:
       is there a reason we only catch these 2?  You think any others are unknown, I'm assuming that will cause the executor to exit?

##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
##########
@@ -113,6 +113,14 @@ class NettyBlockRpcServer(
             s"when there is not sufficient space available to store the block.")
           responseContext.onFailure(exception)
         }
+
+      case req: GetLocalDirsForExecutors =>

Review comment:
       might be nice to rename req getLocalDirs to keep convention like other cases

##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -478,30 +475,43 @@ final class ShuffleBlockFetcherIterator(
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
       val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        hostLocalBlocksByExecutor.keysIterator
+          .filter(exec => execIdsWithoutDirs.contains(exec.executorId))

Review comment:
       exec here is really mId, perhaps rename to be clear.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680418240


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663397122


   **[Test build #126487 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126487/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661672706


   **[Test build #126227 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126227/testReport)** for PR 28911 at commit [`bcb6012`](https://github.com/apache/spark/commit/bcb60123ef68feacf4d571de47cc99068b8496b8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693638



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+              Arrays.toString(getLocalDirsMessage.execIds), t.getCause());
+            hostLocalDirsCompletable.completeExceptionally(t);
+          }
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          logger.warn("Error trying to get the host local dirs for " +

Review comment:
       ditto for error message.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661692684


   **[Test build #126234 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126234/testReport)** for PR 28911 at commit [`bcb6012`](https://github.com/apache/spark/commit/bcb60123ef68feacf4d571de47cc99068b8496b8).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459649979



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       To enable dynamic allocation, it requires spark.shuffle.service.enabled or spark.dynamicAllocation.shuffleTracking.enabled. So I think users must be in one among the 3 conditions.
   
   Looks like these requirements are redundant?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-662863820


   **[Test build #126397 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126397/testReport)** for PR 28911 at commit [`6449229`](https://github.com/apache/spark/commit/64492298c78a86920c278ab5acfda41cb4ea071f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-667843579






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r448971785



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +
+        s"the same host are read from the disk directly instead of being fetched as remote blocks" +
+        s" over the network.")

Review comment:
       I'm not familiar with cloud env. So I just ask for your confirmation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479690817



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -18,15 +18,32 @@
 package org.apache.spark.network.shuffle;
 
 import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 
 import com.codahale.metrics.MetricSet;
+import org.apache.spark.network.client.RpcResponseCallback;
+import org.apache.spark.network.client.TransportClient;
+import org.apache.spark.network.client.TransportClientFactory;
+import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
+import org.apache.spark.network.shuffle.protocol.GetLocalDirsForExecutors;
+import org.apache.spark.network.shuffle.protocol.LocalDirsForExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;

Review comment:
       Hi, @Ngone51 . Could you follow the community guide for `import` order, please?
   - https://github.com/databricks/scala-style-guide/blame/master/README.md#L403-L408
   
   For Java code, Apache Spark follows Oracle’s Java code conventions and Scala guidelines below. The latter is preferred. And, Apache Spark follows the official Scala style guide and Databricks Scala guide. The latter is preferred.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648574962


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/124454/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683342977


   I’d appreciate a chance to review this on Monday prior to merging.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681541690


   **[Test build #127945 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/127945/testReport)** for PR 28911 at commit [`2aa71f6`](https://github.com/apache/spark/commit/2aa71f659bb3be4ee7a7ae71beaf953fc77c4433).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479696967



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       Thank you for adding this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r460136310



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,62 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id to its
+   *                                 local directories if the request handler replies successfully.
+   *                                 Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";
+    GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds);
+    try {
+      TransportClient client = clientFactory.createClient(host, port);
+      client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() {
+        @Override
+        public void onSuccess(ByteBuffer response) {
+          try {
+            BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response);
+            hostLocalDirsCompletable.complete(
+                    ((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
+          } catch (Throwable t) {
+            logger.warn("Error trying to get the host local dirs for " +
+                            Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service",

Review comment:
       Yeah, it's ok. Thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480089706



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {

Review comment:
       I've combined the two host-local shuffle reading tests(with external shuffle service enabled or disabled) into the [HostLocalShuffleReadingSuite](https://github.com/apache/spark/blob/6b97be552b1d5a78f476c4a97d794c567222d9bf/core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleReadingSuite.scala). Thus, the host-local shuffle reading feature can be tested centralized. Does it look okay to you? @dongjoon-hyun @attilapiros 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661801531






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480095723



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.

Review comment:
       It's a pre-condition which should be guaranteed by the caller. I've changed it to `should be`. Thanks! 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479693064



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";

Review comment:
       If we need this assertion, we had better move `ExternalBlockStoreClient.checkInit` function to here, `BlockStoreClient`, in order to remove the duplication. Since this PR already moves several things, moving `checkInit` looks natural.
   ```java
     protected void checkInit() {
       assert appId != null : "Called before init()";
     }
   ```

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,56 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                 to its local directories if the request handler replies
+   *                                 successfully. Otherwise, it contains a specific error.
+   */
+  public void getHostLocalDirs(
+      String host,
+      int port,
+      String[] execIds,
+      CompletableFuture<Map<String, String[]>> hostLocalDirsCompletable) {
+    assert appId != null : "Called before init()";

Review comment:
       If we need this assertion, we had better move `ExternalBlockStoreClient.checkInit` function to here, `BlockStoreClient`, in order to remove the duplication. Since this PR already moves several things, moving `checkInit` together looks natural.
   ```java
     protected void checkInit() {
       assert appId != null : "Called before init()";
     }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683342858






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648571518


   **[Test build #124454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124454/testReport)** for PR 28911 at commit [`0d62ccb`](https://github.com/apache/spark/commit/0d62ccb24e326c2ace1d7b185e0bf18591a7099e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683356005






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681654515






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479697813



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.matchers.must.Matchers
+import org.scalatest.matchers.should.Matchers._
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {
+  test("read host local shuffle from disk with external shuffle service disabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, true)
+      .set(SHUFFLE_SERVICE_ENABLED, false)
+      .set(DYN_ALLOCATION_ENABLED, false)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+    sc.getConf.get(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(true)
+    sc.env.blockManager.externalShuffleServiceEnabled should equal(false)
+    sc.env.blockManager.hostLocalDirManager.isDefined should equal(true)
+    sc.env.blockManager.blockStoreClient.getClass should equal(classOf[NettyBlockTransferService])
+    TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
+
+    val rdd = sc.parallelize(0 until 1000, 10)
+      .map { i => (i, 1) }
+      .reduceByKey(_ + _)
+
+    assert(rdd.count() === 1000)
+    assert(rdd.count() === 1000)
+
+    val cachedExecutors = rdd.mapPartitions { _ =>
+      SparkEnv.get.blockManager.hostLocalDirManager.map { localDirManager =>
+        localDirManager.getCachedHostLocalDirs.keySet.iterator
+      }.getOrElse(Iterator.empty)
+    }.collect().toSet
+
+    // both executors are caching the dirs of the other one
+    cachedExecutors should equal(sc.getExecutorIds().toSet)

Review comment:
       This line adds the test coverage for caching content.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-682137992






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-683372244


   @Ngone51 The PR description mentions disabled dynamic allocation as requirement but this was changed as result of a review finding. Could you please update it?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663397663






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-681654541


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/127945/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663399910






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-655918176


   Jenkins, retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648257419


   thanks for working on this, was interested in this as well. Can you update the description to include details on your overall approach - where do you get the directories from, etc?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-684476666






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r477307059



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,60 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *

Review comment:
       nit remove extra new line

##########
File path: core/src/main/scala/org/apache/spark/network/BlockDataManager.scala
##########
@@ -27,6 +27,8 @@ import org.apache.spark.storage.{BlockId, ShuffleBlockId, StorageLevel}
 private[spark]
 trait BlockDataManager {
 
+  def getLocalDiskDirs: Array[String]

Review comment:
       would be nice to add small java doc, it should be pretty obvious from name but doesn't hurt.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
##########
@@ -61,4 +78,60 @@ public MetricSet shuffleMetrics() {
     // Return an empty MetricSet by default.
     return () -> Collections.emptyMap();
   }
+
+  /**
+   * Request the local disk directories for executors which are located at the same host with
+   * the current BlockStoreClient(it can be ExternalBlockStoreClient or NettyBlockTransferService).
+   *
+   *
+   * @param host the host of BlockManager or ExternalShuffleService. It's the same with current
+   *             BlockStoreClient.
+   * @param port the port of BlockManager or ExternalShuffleService.
+   * @param execIds a collection of executor Ids, which specifies the target executors that we
+   *                want to get their local directories. There could be multiple executor Ids if
+   *                BlockStoreClient is implemented by ExternalBlockStoreClient since the request
+   *                handler, ExternalShuffleService, can serve multiple executors on the same node.
+   *                Or, only one executor Id if BlockStoreClient is implemented by
+   *                NettyBlockTransferService.
+   * @param hostLocalDirsCompletable a CompletableFuture which contains a map from executor Id
+   *                                to its local directories if the request handler replies

Review comment:
       nit last 2 need to be indented one more space




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459751695



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       I agree. I think we should take these out. In the future we may have different requirements and forget to go change here as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r459897035



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,16 @@ 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 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.")
+      .doc("When 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. Note that for k8s workloads, this only works when nodes are " +
+        "using non-isolated container storage." +
+        s"To enable the feature, one should disable ${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key} first." +
+        " And make sure that one of the following requirements are satisfied:\n" +
+        s"1. external shuffle service is enabled (${SHUFFLE_SERVICE_ENABLED.key});" +
+        s"2. dynamic allocation is disabled (${DYN_ALLOCATION_ENABLED.key});" +
+        s"3. dynamic allocation is enabled with shuffle tracking " +
+        s"(${DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED});")

Review comment:
       Oh yeah... we'd cover all the cases after this PR. I'll remove the requirements.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r452704155



##########
File path: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -478,30 +475,43 @@ final class ShuffleBlockFetcherIterator(
       logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " +
         s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}")
       val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray
-      hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) {
-        case Success(dirs) =>
-          immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) =>
-            blockInfos.takeWhile { case (blockId, _, mapIndex) =>
-              fetchHostLocalBlock(
-                blockId,
-                mapIndex,
-                dirs.get(hostLocalBmId.executorId),
-                hostLocalBmId)
+
+      val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) {
+        val host = blockManager.blockManagerId.host
+        val port = blockManager.externalShuffleServicePort
+        Seq((host, port, immutableHostLocalBlocksWithoutDirs.keys.toArray))
+      } else {
+        hostLocalBlocksByExecutor.keysIterator
+          .filter(exec => execIdsWithoutDirs.contains(exec.executorId))

Review comment:
       corrected to `bmId`, thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-656657077


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r479694949



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -120,34 +120,33 @@ private[spark] class ByteBufferBlockData(
 private[spark] class HostLocalDirManager(
     futureExecutionContext: ExecutionContext,
     cacheSize: Int,
-    externalBlockStoreClient: ExternalBlockStoreClient,
-    host: String,
-    externalShuffleServicePort: Int) extends Logging {
+    blockStoreClient: BlockStoreClient) extends Logging {
 
   private val executorIdToLocalDirsCache =
     CacheBuilder
       .newBuilder()
       .maximumSize(cacheSize)
       .build[String, Array[String]]()
 
-  private[spark] def getCachedHostLocalDirs()
-      : scala.collection.Map[String, Array[String]] = executorIdToLocalDirsCache.synchronized {
-    import scala.collection.JavaConverters._
-    return executorIdToLocalDirsCache.asMap().asScala
-  }
+  private[spark] def getCachedHostLocalDirs: Map[String, Array[String]] =
+   executorIdToLocalDirsCache.synchronized {
+      executorIdToLocalDirsCache.asMap().asScala.toMap
+   }
 
   private[spark] def getHostLocalDirs(
+      host: String,
+      port: Int,
       executorIds: Array[String])(
-      callback: Try[java.util.Map[String, Array[String]]] => Unit): Unit = {
+      callback: Try[Map[String, Array[String]]] => Unit): Unit = {

Review comment:
       Just a question. Why do we switch from `Java` Map to `Scala` Map in this PR? Is it required?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r456406950



##########
File path: core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
##########
@@ -194,6 +196,45 @@ private[spark] class NettyBlockTransferService(
     result.future
   }
 
+  override def getHostLocalDirs(
+      host: String,
+      port: Int,
+      execIds: Array[String],
+      hostLocalDirsCompletable: CompletableFuture[util.Map[String, Array[String]]]): Unit = {
+    val getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds)
+    try {
+      val client = clientFactory.createClient(host, port)
+      client.sendRpc(getLocalDirsMessage.toByteBuffer, new RpcResponseCallback() {
+        override def onSuccess(response: ByteBuffer): Unit = {
+          try {
+            val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response)
+            hostLocalDirsCompletable.complete(
+              msgObj.asInstanceOf[LocalDirsForExecutors].getLocalDirsByExec)
+          } catch {
+            case t: Throwable =>
+              logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+                t.getCause)
+              hostLocalDirsCompletable.completeExceptionally(t)
+          } finally {
+            client.close()
+          }
+        }
+
+        override def onFailure(t: Throwable): Unit = {
+          logWarning(s"Error trying to get the host local dirs for executor ${execIds.head}",
+            t.getCause)
+          hostLocalDirsCompletable.completeExceptionally(t)
+          client.close()
+        }
+      })
+    } catch {
+      case e: IOException =>

Review comment:
       @Ngone51 this is almost the same as `ExternalBlockStoreClient.getHostLocalDirs` which only contains one line more:
   https://github.com/apache/spark/blob/1126341a680c3a6bea4f792516ea59b54726e11a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java#L199 
   
   Can you refactor this to reuse the same code?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-652989009






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r446965083



##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleFetchSuite.scala
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.shuffle
+
+import org.scalatest.Matchers
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite, TestUtils}
+import org.apache.spark.internal.config._
+import org.apache.spark.network.netty.NettyBlockTransferService
+
+/**
+ * This test suite is used to test host local shuffle reading with external shuffle service disabled
+ */
+class HostLocalShuffleFetchSuite extends SparkFunSuite with Matchers with LocalSparkContext {
+  test("read host local shuffle from disk with external shuffle service disabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, true)
+      .set(SHUFFLE_SERVICE_ENABLED, false)
+      .set(DYN_ALLOCATION_ENABLED, false)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
+    sc.getConf.get(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(true)
+    sc.env.blockManager.externalShuffleServiceEnabled should equal(false)
+    sc.env.blockManager.hostLocalDirManager.isDefined should equal(true)
+    sc.env.blockManager.blockStoreClient.getClass should equal(classOf[NettyBlockTransferService])
+    TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
+
+    val rdd = sc.parallelize(0 until 1000, 10)
+      .map { i => (i, 1) }
+      .reduceByKey(_ + _)
+
+    rdd.count()
+    rdd.count()
+
+    val cachedExecutors = rdd.mapPartitions { _ =>
+      SparkEnv.get.blockManager.hostLocalDirManager.map { localDirManager =>
+        localDirManager.getCachedHostLocalDirs().keySet.iterator
+      }.getOrElse(Iterator.empty)
+    }.collect().toSet
+
+    // both executors are caching the dirs of the other one
+    cachedExecutors should equal(sc.getExecutorIds().toSet)
+
+    // Now Spark will not receive FetchFailed as host local blocks are read from the cached local
+    // disk directly

Review comment:
       Nit: I would remove this comment as I it is coming from:
   https://github.com/apache/spark/blob/dc8d15c4078774dce031096b418695b51fb100ee/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala#L141-L142
   
   But there we unregistered the application from the external shuffle service so it was not able to resolve the blocks any more. And here we have no external shuffle service at all.
   
   ```suggestion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-663397122


   **[Test build #126487 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126487/testReport)** for PR 28911 at commit [`c9828b7`](https://github.com/apache/spark/commit/c9828b739a667311a3a88cf1e6fa68df7bbbe7f2).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-648567102


   @tgravescs updated the description, thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r458103470



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1391,10 +1391,12 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and 1) external " +
+        s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled or 2) ${DYN_ALLOCATION_ENABLED.key}" +
+        s" is disabled), shuffle blocks requested from those block managers which are running on " +

Review comment:
       I mean, it *can* work with this feature enabled(please see my P.S. comment above). And I will update the document about dynamic allocation to:
   
   1. not allow when `spark.dynamicAllocation.enabled=true` && `spark.dynamicAllocation.shuffleTracking.enabled=false`
   
   2. allow when `spark.dynamicAllocation.enabled=true` && `spark.dynamicAllocation.shuffleTracking.enabled=true`
   
   also cc @attilapiros Does it make sense to you?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661676239


   **[Test build #126227 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126227/testReport)** for PR 28911 at commit [`bcb6012`](https://github.com/apache/spark/commit/bcb60123ef68feacf4d571de47cc99068b8496b8).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `abstract class BlockTransferService extends BlockStoreClient `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661676272






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661801531


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-661130435


   Thank you for the review. I'll try to address them tomorrow!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] attilapiros commented on a change in pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #28911:
URL: https://github.com/apache/spark/pull/28911#discussion_r480111222



##########
File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
##########
@@ -1415,10 +1415,11 @@ 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 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.")
+      .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled, 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. Note that for k8s workloads, this only works when nodes are using " +
+        "non-isolated container storage.")

Review comment:
       @Ngone51 on containerized resource manager having a non-isolated container storage won't be enough, as for this feature to work we need to detect this non-isolation. Currently this is done by using the some host in the blockmanager ID which works only for YARN and standalone mode, is not it?
   A question for the future: do you have a plan to introduce block manager grouping based on shared storage?  
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #28911: [SPARK-32077][CORE] Support host-local shuffle data reading when external shuffle service is disabled

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28911:
URL: https://github.com/apache/spark/pull/28911#issuecomment-680418240






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org