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 2020/07/23 12:57:59 UTC

[GitHub] [spark] yaooqinn opened a new pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

yaooqinn opened a new pull request #29204:
URL: https://github.com/apache/spark/pull/29204


   
   
   ### What changes were proposed in this pull request?
   
   Log error/warn message only once at the server-side for both sync and async modes
   
   ### Why are the changes needed?
   
   In https://github.com/apache/spark/commit/b151194299f5ba15e0d9d8d7d2980fd164fe0822 we make the error logging for  SparkExecuteStatementOperation with `runInBackground=true` not duplicated, but the operations with runInBackground=false and other metadata operation still will be log twice which happened in the operation's `runInternal` method and ThriftCLIService.
   
   In this PR, I propose to reflect the logic to get a unified error handling approach.
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   Yes, when spark.sql.hive.thriftServer.async=false and people call sync APIs the error message will be logged only once at server-side.
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   locally verified the result in target/unit-test.log
   
   add unit tests.
   


----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664106996






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663367974






----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126479 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126479/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -35,7 +37,7 @@ private[hive] trait SparkOperation extends Operation with Logging {
 
   protected def sqlContext: SQLContext
 
-  protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString()
+  protected val statementId: String = getHandle.getHandleIdentifier.toString

Review comment:
       HandleIdentifer.publicId is actually  UUID.randomUUID() too




----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663339197


   **[Test build #126463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126463/testReport)** for PR 29204 at commit [`2726f48`](https://github.com/apache/spark/commit/2726f489fb5c2d3088c43adc24e1721f31b15cbe).


----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663390069






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
##########
@@ -181,7 +178,7 @@ private[hive] class SparkExecuteStatementOperation(
         resultOffset += 1
       }
       previousFetchEndOffset = resultOffset
-      log.info(s"Returning result set with ${curRow} rows from offsets " +
+      logInfo(s"Returning result set with ${curRow} rows from offsets " +

Review comment:
       is `log.info` and `logInfo` the same?




----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126622 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126622/testReport)** for PR 29204 at commit [`dae9bec`](https://github.com/apache/spark/commit/dae9bece4f8f89bd5ff41646c3c3bd349ead53e3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664100091


   **[Test build #126606 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126606/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).


----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664100357






----------------------------------------------------------------
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:
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 pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   retest this please


----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126411/testReport)** for PR 29204 at commit [`91e8f28`](https://github.com/apache/spark/commit/91e8f28bf23f2ee6aa7b473f959485228e4f41ed).


----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664244632






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663341063


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664174550






----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126606 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126606/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664161382


   **[Test build #126622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126622/testReport)** for PR 29204 at commit [`dae9bec`](https://github.com/apache/spark/commit/dae9bece4f8f89bd5ff41646c3c3bd349ead53e3).


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -94,15 +96,32 @@ private[hive] trait SparkOperation extends Operation with Logging {
       throw new IllegalArgumentException(s"Unknown table type is found: $t")
   }
 
-  protected def onError(): PartialFunction[Throwable, Unit] = {
+  protected def onError(needCancel: Boolean = false): PartialFunction[Throwable, Unit] = {
+    // Actually do need to catch Throwable as some failures don't inherit from Exception and
+    // HiveServer will silently swallow them.
     case e: Throwable =>
-      logError(s"Error operating $getType with $statementId", e)
-      super.setState(OperationState.ERROR)
-      HiveThriftServer2.eventManager.onStatementError(
-        statementId, e.getMessage, Utils.exceptionString(e))
-      e match {
-        case _: HiveSQLException => throw e
-        case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e)
+      // When cancel() or close() is called very quickly after the query is started,
+      // then they may both call cleanup() before Spark Jobs are started. But before background
+      // task interrupted, it may have start some spark job, so we need to cancel again to
+      // make sure job was cancelled when background thread was interrupted
+      if (needCancel) sqlContext.sparkContext.cancelJobGroup(statementId)
+      val currentState = getStatus.getState
+      if (currentState.isTerminal) {
+        // This may happen if the execution was cancelled, and then closed from another thread.
+        logWarning(s"Ignore exception in terminal state with $statementId: $e")
+      } else {
+        super.setState(OperationState.ERROR)
+        HiveThriftServer2.eventManager.onStatementError(
+          statementId, e.getMessage, Utils.exceptionString(e))
+        e match {
+          case _: HiveSQLException => throw e
+          case rejected: RejectedExecutionException =>
+            throw new HiveSQLException("The background threadpool cannot accept" +
+              " new task for execution, please retry the operation", rejected)
+          case _ =>
+            val tips = if (shouldRunAsync()) " in background" else ""
+            throw new HiveSQLException(s"Error operating $getType$tips: ${e.getMessage}", e)

Review comment:
       it looks like we missed some details:
   1. Shall we expose the internal details (`getType`) to users or just say `Error running query:`?
   2. Shall we include the full exception string (`e.toString` like https://github.com/apache/spark/pull/29204/files#diff-72dcd8f81a51c8a815159fdf0332acdcL316), not just the message?
   
   Since the PR is small, can we revert and resend it after proper discussion?




----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664209778






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126633/testReport)** for PR 29204 at commit [`5011314`](https://github.com/apache/spark/commit/50113145de4d9c7247d2a8af6e1e4f1087d19548).


----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126633 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126633/testReport)** for PR 29204 at commit [`5011314`](https://github.com/apache/spark/commit/50113145de4d9c7247d2a8af6e1e4f1087d19548).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -35,7 +37,7 @@ private[hive] trait SparkOperation extends Operation with Logging {
 
   protected def sqlContext: SQLContext
 
-  protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString()
+  protected val statementId: String = getHandle.getHandleIdentifier.toString

Review comment:
       Can we revert this change? I see places that use `statementId = UUID.randomUUID().toString`. It's better to have a separated PR to unify this part.




----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126463 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126463/testReport)** for PR 29204 at commit [`2726f48`](https://github.com/apache/spark/commit/2726f489fb5c2d3088c43adc24e1721f31b15cbe).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663341067


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126463/
   Test FAILed.


----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126411 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126411/testReport)** for PR 29204 at commit [`91e8f28`](https://github.com/apache/spark/commit/91e8f28bf23f2ee6aa7b473f959485228e4f41ed).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126622/testReport)** for PR 29204 at commit [`dae9bec`](https://github.com/apache/spark/commit/dae9bece4f8f89bd5ff41646c3c3bd349ead53e3).


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -35,7 +37,7 @@ private[hive] trait SparkOperation extends Operation with Logging {
 
   protected def sqlContext: SQLContext
 
-  protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString()
+  protected val statementId: String = getHandle.getHandleIdentifier.toString

Review comment:
       I will revert this change in this pr




----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126606 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126606/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
##########
@@ -238,21 +237,7 @@ private[hive] class SparkExecuteStatementOperation(
         val backgroundHandle =
           parentSession.getSessionManager().submitBackgroundOperation(backgroundOperation)
         setBackgroundHandle(backgroundHandle)
-      } catch {
-        case rejected: RejectedExecutionException =>
-          logError("Error submitting query in background, query rejected", rejected)
-          setState(OperationState.ERROR)
-          HiveThriftServer2.eventManager.onStatementError(
-            statementId, rejected.getMessage, SparkUtils.exceptionString(rejected))
-          throw new HiveSQLException("The background threadpool cannot accept" +
-            " new task for execution, please retry the operation", rejected)
-        case NonFatal(e) =>
-          logError(s"Error executing query in background", e)

Review comment:
       shall we keep the message `... in background` in https://github.com/apache/spark/pull/29204/files#diff-424526b50bfb53733a4c2e6c6a3ddd8dR121 ?




----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664212962


   **[Test build #126633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126633/testReport)** for PR 29204 at commit [`5011314`](https://github.com/apache/spark/commit/50113145de4d9c7247d2a8af6e1e4f1087d19548).


----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663009833






----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   cc @juliuszsompolski 


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
##########
@@ -238,21 +237,7 @@ private[hive] class SparkExecuteStatementOperation(
         val backgroundHandle =
           parentSession.getSessionManager().submitBackgroundOperation(backgroundOperation)
         setBackgroundHandle(backgroundHandle)
-      } catch {
-        case rejected: RejectedExecutionException =>
-          logError("Error submitting query in background, query rejected", rejected)
-          setState(OperationState.ERROR)
-          HiveThriftServer2.eventManager.onStatementError(
-            statementId, rejected.getMessage, SparkUtils.exceptionString(rejected))
-          throw new HiveSQLException("The background threadpool cannot accept" +
-            " new task for execution, please retry the operation", rejected)
-        case NonFatal(e) =>
-          logError(s"Error executing query in background", e)

Review comment:
       shall we keep the message `Error executing query in background`?




----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-662992287


   **[Test build #126411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126411/testReport)** for PR 29204 at commit [`91e8f28`](https://github.com/apache/spark/commit/91e8f28bf23f2ee6aa7b473f959485228e4f41ed).


----------------------------------------------------------------
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:
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] SparkQA removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663378546


   **[Test build #126479 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126479/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).


----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-664160425






----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-663324645






----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   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.

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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] gatorsmile commented on a change in pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -94,15 +96,32 @@ private[hive] trait SparkOperation extends Operation with Logging {
       throw new IllegalArgumentException(s"Unknown table type is found: $t")
   }
 
-  protected def onError(): PartialFunction[Throwable, Unit] = {
+  protected def onError(needCancel: Boolean = false): PartialFunction[Throwable, Unit] = {
+    // Actually do need to catch Throwable as some failures don't inherit from Exception and
+    // HiveServer will silently swallow them.
     case e: Throwable =>
-      logError(s"Error operating $getType with $statementId", e)
-      super.setState(OperationState.ERROR)
-      HiveThriftServer2.eventManager.onStatementError(
-        statementId, e.getMessage, Utils.exceptionString(e))
-      e match {
-        case _: HiveSQLException => throw e
-        case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e)
+      // When cancel() or close() is called very quickly after the query is started,
+      // then they may both call cleanup() before Spark Jobs are started. But before background
+      // task interrupted, it may have start some spark job, so we need to cancel again to
+      // make sure job was cancelled when background thread was interrupted
+      if (needCancel) sqlContext.sparkContext.cancelJobGroup(statementId)
+      val currentState = getStatus.getState
+      if (currentState.isTerminal) {
+        // This may happen if the execution was cancelled, and then closed from another thread.
+        logWarning(s"Ignore exception in terminal state with $statementId: $e")
+      } else {
+        super.setState(OperationState.ERROR)
+        HiveThriftServer2.eventManager.onStatementError(
+          statementId, e.getMessage, Utils.exceptionString(e))
+        e match {
+          case _: HiveSQLException => throw e
+          case rejected: RejectedExecutionException =>
+            throw new HiveSQLException("The background threadpool cannot accept" +
+              " new task for execution, please retry the operation", rejected)
+          case _ =>
+            val tips = if (shouldRunAsync()) " in background" else ""
+            throw new HiveSQLException(s"Error operating $getType$tips: ${e.getMessage}", e)

Review comment:
       +1




----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126479 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126479/testReport)** for PR 29204 at commit [`5663f98`](https://github.com/apache/spark/commit/5663f98d1982dfbd500aa72a0d2394c9b9832a82).


----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala
##########
@@ -94,15 +96,32 @@ private[hive] trait SparkOperation extends Operation with Logging {
       throw new IllegalArgumentException(s"Unknown table type is found: $t")
   }
 
-  protected def onError(): PartialFunction[Throwable, Unit] = {
+  protected def onError(needCancel: Boolean = false): PartialFunction[Throwable, Unit] = {
+    // Actually do need to catch Throwable as some failures don't inherit from Exception and
+    // HiveServer will silently swallow them.
     case e: Throwable =>
-      logError(s"Error operating $getType with $statementId", e)
-      super.setState(OperationState.ERROR)
-      HiveThriftServer2.eventManager.onStatementError(
-        statementId, e.getMessage, Utils.exceptionString(e))
-      e match {
-        case _: HiveSQLException => throw e
-        case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e)
+      // When cancel() or close() is called very quickly after the query is started,
+      // then they may both call cleanup() before Spark Jobs are started. But before background
+      // task interrupted, it may have start some spark job, so we need to cancel again to
+      // make sure job was cancelled when background thread was interrupted
+      if (needCancel) sqlContext.sparkContext.cancelJobGroup(statementId)
+      val currentState = getStatus.getState
+      if (currentState.isTerminal) {
+        // This may happen if the execution was cancelled, and then closed from another thread.
+        logWarning(s"Ignore exception in terminal state with $statementId: $e")
+      } else {
+        super.setState(OperationState.ERROR)
+        HiveThriftServer2.eventManager.onStatementError(
+          statementId, e.getMessage, Utils.exceptionString(e))
+        e match {
+          case _: HiveSQLException => throw e
+          case rejected: RejectedExecutionException =>
+            throw new HiveSQLException("The background threadpool cannot accept" +
+              " new task for execution, please retry the operation", rejected)
+          case _ =>
+            val tips = if (shouldRunAsync()) "in background" else ""

Review comment:
       `val tips = if (shouldRunAsync()) " in background" else ""` and `...$getType$tips ${e.getMessage}`, to avoid extra space.




----------------------------------------------------------------
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:
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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
##########
@@ -238,21 +237,7 @@ private[hive] class SparkExecuteStatementOperation(
         val backgroundHandle =
           parentSession.getSessionManager().submitBackgroundOperation(backgroundOperation)
         setBackgroundHandle(backgroundHandle)
-      } catch {
-        case rejected: RejectedExecutionException =>
-          logError("Error submitting query in background, query rejected", rejected)
-          setState(OperationState.ERROR)
-          HiveThriftServer2.eventManager.onStatementError(
-            statementId, rejected.getMessage, SparkUtils.exceptionString(rejected))
-          throw new HiveSQLException("The background threadpool cannot accept" +
-            " new task for execution, please retry the operation", rejected)
-        case NonFatal(e) =>
-          logError(s"Error executing query in background", e)

Review comment:
       Yes, we can keep 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.

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 #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
##########
@@ -181,7 +178,7 @@ private[hive] class SparkExecuteStatementOperation(
         resultOffset += 1
       }
       previousFetchEndOffset = resultOffset
-      log.info(s"Returning result set with ${curRow} rows from offsets " +
+      logInfo(s"Returning result set with ${curRow} rows from offsets " +

Review comment:
       `logInfo` will check `log.isInfoEnabled`




----------------------------------------------------------------
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:
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 pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   gentle ping @cloud-fan


----------------------------------------------------------------
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:
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 closed pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29204:
URL: https://github.com/apache/spark/pull/29204


   


----------------------------------------------------------------
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:
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] AmplabJenkins commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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






----------------------------------------------------------------
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:
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] AmplabJenkins removed a comment on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29204:
URL: https://github.com/apache/spark/pull/29204#issuecomment-662992881






----------------------------------------------------------------
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:
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] SparkQA commented on pull request #29204: [SPARK-32412][SQL] Unify error handling for spark thrift server operations

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


   **[Test build #126463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126463/testReport)** for PR 29204 at commit [`2726f48`](https://github.com/apache/spark/commit/2726f489fb5c2d3088c43adc24e1721f31b15cbe).


----------------------------------------------------------------
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:
users@infra.apache.org



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