mridulm commented on a change in pull request #28618:
URL: https://github.com/apache/spark/pull/28618#discussion_r473596267



##########
File path: 
core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.api.metadata;
+
+/**
+ * :: Private ::
+ *
+ * A plugin that can monitor the storage of shuffle data from map tasks, and 
can provide
+ * metadata to shuffle readers to aid their reading of shuffle blocks in 
reduce tasks.
+ * <p>
+ * {@link MapOutputMetadata} instances provided from the plugin tree's 
implementation of
+ * {@link org.apache.spark.shuffle.api.ShuffleMapOutputWriter} are sent to
+ * <p>
+ * Implementations MUST be thread-safe. Spark will invoke methods in this 
module in parallel.
+ */
+public interface ShuffleOutputTracker {
+
+  /**
+   * Called when a new shuffle stage is going to be run.
+   */
+  void registerShuffle(int shuffleId);
+

Review comment:
       A callback to signal completion (or failure) of shuffle for a 
`shuffleId` will help `ShuffleDriverComponents` coordinate with external 
services to commit (or cleanup) shuffle metadata/files.

##########
File path: core/src/main/scala/org/apache/spark/ContextCleaner.scala
##########
@@ -222,15 +220,10 @@ private[spark] class ContextCleaner(
   /** Perform shuffle cleanup. */
   def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = {
     try {
-      if (mapOutputTrackerMaster.containsShuffle(shuffleId)) {
-        logDebug("Cleaning shuffle " + shuffleId)
-        mapOutputTrackerMaster.unregisterShuffle(shuffleId)
-        shuffleDriverComponents.removeShuffle(shuffleId, blocking)
-        listeners.asScala.foreach(_.shuffleCleaned(shuffleId))
-        logDebug("Cleaned shuffle " + shuffleId)
-      } else {
-        logDebug("Asked to cleanup non-existent shuffle (maybe it was already 
removed)")
-      }
+      logDebug("Cleaning shuffle " + shuffleId)

Review comment:
       Why was the containsShuffle check removed ?

##########
File path: 
core/src/main/java/org/apache/spark/shuffle/api/metadata/ShuffleOutputTracker.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.api.metadata;
+
+/**
+ * :: Private ::
+ *
+ * A plugin that can monitor the storage of shuffle data from map tasks, and 
can provide
+ * metadata to shuffle readers to aid their reading of shuffle blocks in 
reduce tasks.
+ * <p>
+ * {@link MapOutputMetadata} instances provided from the plugin tree's 
implementation of
+ * {@link org.apache.spark.shuffle.api.ShuffleMapOutputWriter} are sent to
+ * <p>
+ * Implementations MUST be thread-safe. Spark will invoke methods in this 
module in parallel.

Review comment:
       Add a note about what locking semantics are when the methods are invoked 
?

##########
File path: 
core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java
##########
@@ -17,69 +17,64 @@
 
 package org.apache.spark.shuffle.sort.io;
 
-import java.util.Map;
 import java.util.Optional;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 
 import org.apache.spark.SparkConf;
 import org.apache.spark.SparkEnv;
+import org.apache.spark.shuffle.IndexShuffleBlockResolver;
 import org.apache.spark.shuffle.api.ShuffleExecutorComponents;
 import org.apache.spark.shuffle.api.ShuffleMapOutputWriter;
-import org.apache.spark.shuffle.IndexShuffleBlockResolver;
 import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter;
-import org.apache.spark.storage.BlockManager;
 
 public class LocalDiskShuffleExecutorComponents implements 
ShuffleExecutorComponents {
 
   private final SparkConf sparkConf;
-  private BlockManager blockManager;
-  private IndexShuffleBlockResolver blockResolver;
+  private final Supplier<IndexShuffleBlockResolver> blockResolver;

Review comment:
       Here and elsewhere, why rely on `Supplier` when output is memoized ?
   Replace with instance instead ?

##########
File path: 
core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java
##########
@@ -35,12 +37,13 @@ public LocalDiskShuffleDataIO(SparkConf sparkConf) {
   }
 
   @Override
-  public ShuffleExecutorComponents executor() {
-    return new LocalDiskShuffleExecutorComponents(sparkConf);
+  public ShuffleDriverComponents initializeShuffleDriverComponents() {
+    return new LocalDiskShuffleDriverComponents(sparkConf);
   }
 
   @Override
-  public ShuffleDriverComponents driver() {
-    return new LocalDiskShuffleDriverComponents();
+  public ShuffleExecutorComponents initializeShuffleExecutorComponents(
+      String appId, String execId, Map<String, String> extraConfigs) {
+    return new LocalDiskShuffleExecutorComponents(sparkConf);

Review comment:
       nit: ignoring `extraConfigs` - assert it is empty ?




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to