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 2022/02/21 08:32:27 UTC

[GitHub] [spark] AngersZhuuuu opened a new pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

AngersZhuuuu opened a new pull request #35594:
URL: https://github.com/apache/spark/pull/35594


   ### What changes were proposed in this pull request?
   Current Spark SQL CLI alway use  shutdown hook to stop SparkSQLEnv
   ```
    // Clean up after we exit
       ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop() }
   ```
   
   but use process ret to close client side jvm
   ```
   while (line != null) {
     if (!line.startsWith("--")) {
       if (prefix.nonEmpty) {
         prefix += '\n'
        } 
       if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) {
         line = prefix + line
         ret = cli.processLine(line, true)
         prefix = ""
         currentPrompt = promptWithCurrentDB
       } else {
         prefix = prefix + line
         currentPrompt = continuedPromptWithDBSpaces
       }
     }
     line = reader.readLine(currentPrompt + "> ")
   }
   
   sessionState.close()
   
   System.exit(ret)
   }
   ```
   
   ```
       if (sessionState.execString != null) {
         exitCode = cli.processLine(sessionState.execString)
         System.exit(exitCode)
       }
   
       try {
         if (sessionState.fileName != null) {
           exitCode = cli.processFile(sessionState.fileName)
           System.exit(exitCode)
         }
       } catch {
         case e: FileNotFoundException =>
           logError(s"Could not open input file for reading. (${e.getMessage})")
           exitCode = 3
           System.exit(exitCode)
       }
   
   ```
   
   This always cause client side exit code not consistent with AM.
   
   IN this pr I prefer to pass the exit code to `SparkContext.stop()` method to pass a clear cliet side status to AM side in client mode. And I think this can benefit many similar case in future.
   
   ### Why are the changes needed?
   Keep client side status consistent  with AM side
   
   
   ### Does this PR introduce _any_ user-facing change?
   With this pr, client side status will be same with am side
   
   
   ### How was this patch tested?
   MT
   


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] yaooqinn commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -2059,7 +2059,20 @@ class SparkContext(config: SparkConf) extends Logging {
   /**
    * Shut down the SparkContext.
    */
-  def stop(): Unit = {
+  def stop(): Unit = stop(0)
+
+  /**
+   * Shut down the SparkContext with exit code that will passed to scheduler backend.
+   * In client mode, client side may call `SparkContext.stop()` to clean up but exit with
+   * code not equal to 0. This behavior cause resource scheduler such as `ApplicationMaster`
+   * exit with success status but client side exited with failed status. Spark can call
+   * this method to stop SparkContext and pass client side correct exit code to scheduler backend.
+   * Then scheduler backend should send the exit code to corresponding resource scheduler
+   * to keep consistens.

Review comment:
       consistent




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   gentle ping @bogdanghit 


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
##########
@@ -846,8 +850,13 @@ private[spark] class ApplicationMaster(
       // This avoids potentially reporting incorrect exit codes if the driver fails
       if (!(isClusterMode || sparkConf.get(YARN_UNMANAGED_AM))) {
         if (shutdown || !clientModeTreatDisconnectAsFailed) {
-          logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
-          finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+          if (exitCode == 0) {
+            logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
+            finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+          } else {
+            logInfo(s"Driver terminated with exit code ${exitCode}! Shutting down. $remoteAddress")

Review comment:
       > logError
   
   DOne




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] LuciferYang commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -139,7 +145,7 @@ private[hive] object SparkSQLCLIDriver extends Logging {
     SessionState.setCurrentSessionState(sessionState)
 
     // Clean up after we exit
-    ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop() }
+    ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop(exitCode) }

Review comment:
       ~~ShutdownHook is executed on another thread,  `exitCode` should be `volatile`~~
   
   Ignore 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.

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

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] cloud-fan commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35594:
URL: https://github.com/apache/spark/pull/35594#issuecomment-1047511492


   cc @bogdanghit 


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] yaooqinn commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
##########
@@ -846,8 +850,13 @@ private[spark] class ApplicationMaster(
       // This avoids potentially reporting incorrect exit codes if the driver fails
       if (!(isClusterMode || sparkConf.get(YARN_UNMANAGED_AM))) {
         if (shutdown || !clientModeTreatDisconnectAsFailed) {
-          logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
-          finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+          if (exitCode == 0) {
+            logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
+            finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+          } else {
+            logInfo(s"Driver terminated with exit code ${exitCode}! Shutting down. $remoteAddress")

Review comment:
       logError




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] LuciferYang commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   > IN this pr I prefer to pass the exit code to SparkContext.stop() method to pass a clear cliet side status to AM side in client mode. And I think this can benefit many similar case in future.
   
   cliet -> client


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   > > IN this pr I prefer to pass the exit code to SparkContext.stop() method to pass a clear cliet side status to AM side in client mode. And I think this can benefit many similar case in future.
   > 
   > cliet -> client
   
   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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -2059,7 +2059,9 @@ class SparkContext(config: SparkConf) extends Logging {
   /**
    * Shut down the SparkContext.
    */
-  def stop(): Unit = {
+  def stop(): Unit = stop(0)
+
+  def stop(exitCode: Int): Unit = {

Review comment:
       > This is a public API and we need to document it properly.
   
   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-unsubscribe@spark.apache.org

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] bogdanghit commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   @AngersZhuuuu does this address the CliSuite flakiness I reported a while back?


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] LuciferYang commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -139,7 +145,7 @@ private[hive] object SparkSQLCLIDriver extends Logging {
     SessionState.setCurrentSessionState(sessionState)
 
     // Clean up after we exit
-    ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop() }
+    ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop(exitCode) }

Review comment:
       ShutdownHook is executed on another thread,  `exitCode` should be `volatile`
   




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] LuciferYang commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -476,7 +482,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           // Kill the VM on second ctrl+c
           if (!initialRequest) {
             console.printInfo("Exiting the JVM")
-            System.exit(127)
+            SparkSQLCLIDriver.exit(127)

Review comment:
       Although not introduced by this pr, but I think we should give `127` a definite name.
   
   




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] cloud-fan commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35594:
URL: https://github.com/apache/spark/pull/35594#discussion_r811655935



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -2059,7 +2059,9 @@ class SparkContext(config: SparkConf) extends Logging {
   /**
    * Shut down the SparkContext.
    */
-  def stop(): Unit = {
+  def stop(): Unit = stop(0)
+
+  def stop(exitCode: Int): Unit = {

Review comment:
       This is a public API and we need to document it properly.




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   gentle ping @cloud-fan @tgravescs @mridulm 


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
##########
@@ -476,7 +482,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           // Kill the VM on second ctrl+c
           if (!initialRequest) {
             console.printInfo("Exiting the JVM")
-            System.exit(127)
+            SparkSQLCLIDriver.exit(127)

Review comment:
       > Although not introduced by this pr, but I think we should give `127` a definite name.
   
   I will raise a new pr for all CLI's exit 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.

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

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] AngersZhuuuu commented on a change in pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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



##########
File path: core/src/main/scala/org/apache/spark/SparkContext.scala
##########
@@ -2059,7 +2059,20 @@ class SparkContext(config: SparkConf) extends Logging {
   /**
    * Shut down the SparkContext.
    */
-  def stop(): Unit = {
+  def stop(): Unit = stop(0)
+
+  /**
+   * Shut down the SparkContext with exit code that will passed to scheduler backend.
+   * In client mode, client side may call `SparkContext.stop()` to clean up but exit with
+   * code not equal to 0. This behavior cause resource scheduler such as `ApplicationMaster`
+   * exit with success status but client side exited with failed status. Spark can call
+   * this method to stop SparkContext and pass client side correct exit code to scheduler backend.
+   * Then scheduler backend should send the exit code to corresponding resource scheduler
+   * to keep consistens.

Review comment:
       Done




-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] cloud-fan edited a comment on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35594:
URL: https://github.com/apache/spark/pull/35594#issuecomment-1047511492


   cc @bogdanghit  @yaooqinn 


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   gentle ping @cloud-fan @bogdanghit 


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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] AngersZhuuuu commented on pull request #35594: [SPARK-38270][SQL] Spark SQL CLI's AM should keep same exit code with client side

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


   > @AngersZhuuuu does this address the CliSuite flakiness I reported a while back?
   
   Can you provide the link?  Literally, it should be irrelevant


-- 
This is an automated message from the 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-unsubscribe@spark.apache.org

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