Re: [PR] [SPARK-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


xieshuaihu commented on PR #46278:
URL: https://github.com/apache/spark/pull/46278#issuecomment-2084057782

   @dongjoon-hyun @HyukjinKwon 
   I think this question again. Now I think the point is that **the scheduler 
and pool is set by the server, but the client don't know**. 
   
   So I will define a new rpc which get scheduler info from server and this 
info will be cached in client. Then if in FIFO fashino or the pool don't exist, 
call setSchedulerPool will throw exception directly.
   
   Please let me know if this is a good solution?


-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


dongjoon-hyun commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1583392857


##
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##
@@ -808,6 +808,33 @@ class SparkSession private[sql] (
 client.clearTags()
   }
 
+  /**
+   * Set the scheduler pool used to execute all the operations by this thread 
in this session.
+   *
+   * Spark supports FIFO scheduler (default) and Fair scheduler. When fair 
scheduler is enabled,
+   * jobs will be grouped into pools and each pool gets an assigned share of 
cluster resources.
+   * Application programmers can use this method to assign a pool to all the 
operations started
+   * by this thread in this session. This pool should exists in the server. If 
not, will
+   * fallback to FIFO scheduler.
+   *
+   * Note: this doesn't make sense for FIFO scheduler.

Review Comment:
   ? Shall we remove this simply?
   
   We understand that this is not for FIFO and it fallback to FIFO already.



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


dongjoon-hyun commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1583392857


##
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##
@@ -808,6 +808,33 @@ class SparkSession private[sql] (
 client.clearTags()
   }
 
+  /**
+   * Set the scheduler pool used to execute all the operations by this thread 
in this session.
+   *
+   * Spark supports FIFO scheduler (default) and Fair scheduler. When fair 
scheduler is enabled,
+   * jobs will be grouped into pools and each pool gets an assigned share of 
cluster resources.
+   * Application programmers can use this method to assign a pool to all the 
operations started
+   * by this thread in this session. This pool should exists in the server. If 
not, will
+   * fallback to FIFO scheduler.
+   *
+   * Note: this doesn't make sense for FIFO scheduler.

Review Comment:
   ? Shall we remove this simply?



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


HyukjinKwon commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582873705


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   So, you run a Spark Connect server, and only the first Spark Session that an 
user creates can set the scheduler? We block all the static configurations in 
Spark Connect for now.



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


xieshuaihu commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582847360


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   No, scheduler mode can't be changed at runtime. 
   
   But I think in multi-user environment, fair scheduler and pool is better, 
and users need a way to set it's pool



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


HyukjinKwon commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582843351


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   Server concept configuration should not be added to the client. Can you 
change scheduler mode at runtime?



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


xieshuaihu commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582838182


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   I commented in this api. This api should cowork with server.
   
![image](https://github.com/apache/spark/assets/24190835/fa43a927-f5c3-4553-9a2b-bdd2bd41bd7a)
   
   I use this feature in our spark environment, it works.



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


HyukjinKwon commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582831117


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   It has never been tested because it was impossible for the end users to set. 
Does it even work?



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


xieshuaihu commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582815755


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   `conf.set("spark.scheduler.mode", "FAIR")` is a core conf, but 
`SparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)` is not



-- 
This is an automated message from the 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-48040][CONNECT]Spark connect supports scheduler pool [spark]

2024-04-29 Thread via GitHub


HyukjinKwon commented on code in PR #46278:
URL: https://github.com/apache/spark/pull/46278#discussion_r1582812044


##
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##
@@ -177,6 +177,10 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
 s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
   session.sparkContext.setInterruptOnCancel(true)
 
+  executeHolder.schedulerPool.foreach { pool =>
+
session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)

Review Comment:
   This is a core conf so can't b set runtime. This has to be set when users 
run the Spark Connect server.



-- 
This is an automated message from the 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