Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


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


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {
+  case ae: AdaptiveSparkPlanExec =>
+ae.context.shuffleIds.asScala.keys
+  case _ =>
+Iterable.empty
+}
+shuffleIds.foreach { shuffleId =>
+  queryExecution.shuffleCleanupMode match {
+case RemoveShuffleFiles =>
+  SparkEnv.get.shuffleManager.unregisterShuffle(shuffleId)

Review Comment:
   Created https://github.com/apache/spark/pull/46302.



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1584496133


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {

Review Comment:
   Ideally we should clean up shuffles for CTAS and INSERT as well, so they 
also run queries.



##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {

Review Comment:
   Ideally we should clean up shuffles for CTAS and INSERT as well, as they 
also run queries.



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


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


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {
+  case ae: AdaptiveSparkPlanExec =>
+ae.context.shuffleIds.asScala.keys
+  case _ =>
+Iterable.empty
+}
+shuffleIds.foreach { shuffleId =>
+  queryExecution.shuffleCleanupMode match {
+case RemoveShuffleFiles =>
+  SparkEnv.get.shuffleManager.unregisterShuffle(shuffleId)

Review Comment:
   Thanks for catching this! Will fix this in a follow-up asap.



##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {

Review Comment:
   I could be wrong but I thought `DataFrame`s for commands are created in 
`SparkConnectPlanner`, and the ones for queries are only created in 
`SparkConnectPlanExecution`?



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1584298803


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {

Review Comment:
   Oh this is a good catch! I think we should. cc @bozhang2820 



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


ulysses-you commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1584239353


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {

Review Comment:
   It seems the root node can be a command. Shall we collect all the 
AdaptiveSparkPlanExec inside the plan ?



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-30 Thread via GitHub


ulysses-you commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1584223064


##
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##
@@ -161,6 +165,24 @@ object SQLExecution extends Logging {
 case e =>
   Utils.exceptionString(e)
   }
+  if (queryExecution.shuffleCleanupMode != DoNotCleanup
+&& isExecutedPlanAvailable) {
+val shuffleIds = queryExecution.executedPlan match {
+  case ae: AdaptiveSparkPlanExec =>
+ae.context.shuffleIds.asScala.keys
+  case _ =>
+Iterable.empty
+}
+shuffleIds.foreach { shuffleId =>
+  queryExecution.shuffleCleanupMode match {
+case RemoveShuffleFiles =>
+  SparkEnv.get.shuffleManager.unregisterShuffle(shuffleId)

Review Comment:
   Shall we call `shuffleDriverComponents.removeShuffle` ? We are at driver 
side, `shuffleManager.unregisterShuffle` would do nothing in non-local mode.



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-24 Thread via GitHub


cloud-fan closed pull request #45930: [SPARK-47764][CORE][SQL] Cleanup shuffle 
dependencies based on ShuffleCleanupMode
URL: https://github.com/apache/spark/pull/45930


-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-24 Thread via GitHub


cloud-fan commented on PR #45930:
URL: https://github.com/apache/spark/pull/45930#issuecomment-2074349569

   thanks, merging to master!


-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


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


##
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala:
##
@@ -869,6 +874,8 @@ case class AdaptiveExecutionContext(session: SparkSession, 
qe: QueryExecution) {
*/
   val stageCache: TrieMap[SparkPlan, ExchangeQueryStageExec] =
 new TrieMap[SparkPlan, ExchangeQueryStageExec]()
+
+  val shuffleIds: TrieMap[Int, Boolean] = new TrieMap[Int, Boolean]()

Review Comment:
   Updated



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


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


##
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala:
##
@@ -187,6 +187,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) 
extends ShuffleManager
 shuffleBlockResolver.removeDataByMap(shuffleId, mapTaskId)
   }
 }
+shuffleBlockResolver.removeShuffleToSkip(shuffleId)

Review Comment:
   Yeah this is a bit weird... Changed to use a Guava cache with a fixed 
maximum size (1000) instead, so that we do not need to do cleanups for 
shufflesToSkip. 



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1576289113


##
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala:
##
@@ -869,6 +874,8 @@ case class AdaptiveExecutionContext(session: SparkSession, 
qe: QueryExecution) {
*/
   val stageCache: TrieMap[SparkPlan, ExchangeQueryStageExec] =
 new TrieMap[SparkPlan, ExchangeQueryStageExec]()
+
+  val shuffleIds: TrieMap[Int, Boolean] = new TrieMap[Int, Boolean]()

Review Comment:
   yea, concurrent hash map with int key should be good 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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


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


##
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala:
##
@@ -187,6 +187,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) 
extends ShuffleManager
 shuffleBlockResolver.removeDataByMap(shuffleId, mapTaskId)
   }
 }
+shuffleBlockResolver.removeShuffleToSkip(shuffleId)

Review Comment:
   Yeah this is a bit weird... Changed to use a Guava cache with a fixed 
maximum size (1000). 



##
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala:
##
@@ -869,6 +874,8 @@ case class AdaptiveExecutionContext(session: SparkSession, 
qe: QueryExecution) {
*/
   val stageCache: TrieMap[SparkPlan, ExchangeQueryStageExec] =
 new TrieMap[SparkPlan, ExchangeQueryStageExec]()
+
+  val shuffleIds: TrieMap[Int, Boolean] = new TrieMap[Int, Boolean]()

Review Comment:
   I think a concurrent hash map is still required since the context are shared 
between the main query and all sub queries?



##
core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:
##
@@ -76,13 +78,21 @@ 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 Option(shuffleIdsToSkip.getIfPresent(shuffleId)).isEmpty =>
 Some(ShuffleBlockInfo(shuffleId, mapId))
   case _ =>
 None
 }
   }
 
+  private val shuffleIdsToSkip =
+CacheBuilder.newBuilder().maximumSize(1000).build[java.lang.Integer, 
java.lang.Boolean]()

Review Comment:
   Unfortunately Guava cache won't accept null values...



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1575686724


##
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala:
##
@@ -869,6 +874,8 @@ case class AdaptiveExecutionContext(session: SparkSession, 
qe: QueryExecution) {
*/
   val stageCache: TrieMap[SparkPlan, ExchangeQueryStageExec] =
 new TrieMap[SparkPlan, ExchangeQueryStageExec]()
+
+  val shuffleIds: TrieMap[Int, Boolean] = new TrieMap[Int, Boolean]()

Review Comment:
   what does the value mean? BTW, `stageCache` uses `TrieMap` because the key 
is `SparkPlan`. For int key, I think normal hash map works fine



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-23 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1575684967


##
core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:
##
@@ -76,13 +78,21 @@ 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 Option(shuffleIdsToSkip.getIfPresent(shuffleId)).isEmpty =>
 Some(ShuffleBlockInfo(shuffleId, mapId))
   case _ =>
 None
 }
   }
 
+  private val shuffleIdsToSkip =
+CacheBuilder.newBuilder().maximumSize(1000).build[java.lang.Integer, 
java.lang.Boolean]()

Review Comment:
   if the value does not matter, shall we just use `Object` type and always 
pass null?



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-17 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1569819026


##
core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala:
##
@@ -187,6 +187,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) 
extends ShuffleManager
 shuffleBlockResolver.removeDataByMap(shuffleId, mapTaskId)
   }
 }
+shuffleBlockResolver.removeShuffleToSkip(shuffleId)

Review Comment:
   this is a weird place to do cleanup. Shall we cover all shuffle manager 
implementations? Shall we do it in the caller of this `unregisterShuffle` 
function?



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-17 Thread via GitHub


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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-15 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1565226301


##
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:
   What's the life cycle of 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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-15 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1565222770


##
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:
   can we `new QueryExecution` here? Then we don't need to touch session state 
builder



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-15 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1565220695


##
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:
   isn't this the 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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-15 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1565218933


##
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:
   ```suggestion
 .doc("When enabled, shuffle files will be cleaned up at the end of SQL 
" +
   ```



-- 
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: reviews-unsubscr...@spark.apache.org

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


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



Re: [PR] [SPARK-47764][CORE][SQL] Cleanup shuffle dependencies based on ShuffleCleanupMode [spark]

2024-04-15 Thread via GitHub


cloud-fan commented on code in PR #45930:
URL: https://github.com/apache/spark/pull/45930#discussion_r1565218492


##
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:
   let's add a default implememtation



-- 
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: reviews-unsubscr...@spark.apache.org

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


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