bozhang2820 commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1568946086


##########
core/src/main/scala/org/apache/spark/shuffle/MigratableResolver.scala:
##########
@@ -35,6 +35,11 @@ trait MigratableResolver {
    */
   def getStoredShuffles(): Seq[ShuffleBlockInfo]
 
+  /**
+   * Mark a shuffle that should not be migrated.
+   */
+  def addShuffleToSkip(shuffleId: Int): Unit

Review Comment:
   Done.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -95,10 +95,26 @@ private[sql] object Dataset {
       new Dataset[Row](qe, ExpressionEncoder(qe.analyzed.schema))
   }
 
+  def ofRows(
+      sparkSession: SparkSession,
+      logicalPlan: LogicalPlan,
+      shuffleCleanupMode: ShuffleCleanupMode): DataFrame =
+    sparkSession.withActive {
+      val qe = sparkSession.sessionState.executePlan(

Review Comment:
   Good idea. Done.



##########
core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:
##########
@@ -76,13 +78,19 @@ private[spark] class IndexShuffleBlockResolver(
   override def getStoredShuffles(): Seq[ShuffleBlockInfo] = {
     val allBlocks = blockManager.diskBlockManager.getAllBlocks()
     allBlocks.flatMap {
-      case ShuffleIndexBlockId(shuffleId, mapId, _) =>
+      case ShuffleIndexBlockId(shuffleId, mapId, _) if 
!shuffleIdsToSkip.contains(shuffleId) =>
         Some(ShuffleBlockInfo(shuffleId, mapId))
       case _ =>
         None
     }
   }
 
+  private val shuffleIdsToSkip = Collections.newSetFromMap[Int](new 
ConcurrentHashMap)

Review Comment:
   Updated to remove from this Set when the shuffle is unregistered. 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -2854,6 +2854,23 @@ object SQLConf {
       .intConf
       
.createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
 
+  val SHUFFLE_DEPENDENCY_SKIP_MIGRATION_ENABLED =
+    buildConf("spark.sql.shuffleDependency.skipMigration.enabled")
+      .doc("When enabled, shuffle dependencies for a Spark Connect SQL 
execution are marked at " +
+        "the end of the execution, and they will not be migrated during 
decommissions.")
+      .version("4.0.0")
+      .booleanConf
+      .createWithDefault(Utils.isTesting)
+
+  val SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED =
+    buildConf("spark.sql.shuffleDependency.fileCleanup.enabled")
+      .doc("When enabled, shuffle dependency files will be cleaned up at the 
end of SQL " +

Review Comment:
   Updated.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala:
##########
@@ -108,7 +108,8 @@ class CacheManager extends Logging with 
AdaptiveSparkPlanHelper {
     } else {
       val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark)
       val inMemoryRelation = sessionWithConfigsOff.withActive {
-        val qe = sessionWithConfigsOff.sessionState.executePlan(planToCache)
+        val qe = sessionWithConfigsOff.sessionState.executePlan(
+          planToCache, shuffleCleanupMode = DoNotCleanup)

Review Comment:
   Tried to be explicit here. Removed the unnecessary argument. 



-- 
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.

To unsubscribe, e-mail: [email protected]

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