-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26533][SQL] Support query auto timeout cancel on thriftserver #29933
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
42e3ad2
498272a
e6787db
ec3f043
066d622
3bfc809
3d35de1
a1e2f65
5df36e6
3bdeb7a
2a32d0d
5e3c440
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -893,6 +893,17 @@ object SQLConf { | |
| .booleanConf | ||
| .createWithDefault(false) | ||
|
|
||
| val THRIFTSERVER_QUERY_TIMEOUT = | ||
| buildConf("spark.sql.thriftServer.queryTimeout") | ||
| .doc("Set a query duration timeout in seconds in Thrift Server. If the timeout is set to " + | ||
| "a positive value, a running query will be cancelled automatically when the timeout is " + | ||
| "exceeded, otherwise the query continues to run till completion. If timeout values are " + | ||
| "set for each statement via `java.sql.Statement.setQueryTimeout` and they are smaller " + | ||
| "than this configuration value, they take precedence.") | ||
| .version("3.1.0") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is this version correct?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea, I think so. |
||
| .timeConf(TimeUnit.SECONDS) | ||
| .createWithDefault(0L) | ||
|
|
||
| val THRIFTSERVER_UI_STATEMENT_LIMIT = | ||
| buildConf("spark.sql.thriftserver.ui.retainedStatements") | ||
| .doc("The number of SQL statements kept in the JDBC/ODBC web UI history.") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver | |
|
|
||
| import java.security.PrivilegedExceptionAction | ||
| import java.util.{Arrays, Map => JMap} | ||
| import java.util.concurrent.RejectedExecutionException | ||
| import java.util.concurrent.{Executors, RejectedExecutionException, TimeUnit} | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
@@ -45,11 +45,24 @@ private[hive] class SparkExecuteStatementOperation( | |
| parentSession: HiveSession, | ||
| statement: String, | ||
| confOverlay: JMap[String, String], | ||
| runInBackground: Boolean = true) | ||
| runInBackground: Boolean = true, | ||
| queryTimeout: Long) | ||
| extends ExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground) | ||
| with SparkOperation | ||
| with Logging { | ||
|
|
||
| // If a timeout value `queryTimeout` is specified by users and it is smaller than | ||
| // a global timeout value, we use the user-specified value. | ||
| // This code follows the Hive timeout behaviour (See #29933 for details). | ||
| private val timeout = { | ||
| val globalTimeout = sqlContext.conf.getConf(SQLConf.THRIFTSERVER_QUERY_TIMEOUT) | ||
| if (globalTimeout > 0 && (queryTimeout <= 0 || globalTimeout < queryTimeout)) { | ||
| globalTimeout | ||
| } else { | ||
| queryTimeout | ||
| } | ||
| } | ||
|
|
||
| private var result: DataFrame = _ | ||
|
|
||
| // We cache the returned rows to get iterators again in case the user wants to use FETCH_FIRST. | ||
|
|
@@ -200,6 +213,23 @@ private[hive] class SparkExecuteStatementOperation( | |
| parentSession.getUsername) | ||
| setHasResultSet(true) // avoid no resultset for async run | ||
|
|
||
| if (timeout > 0) { | ||
| val timeoutExecutor = Executors.newSingleThreadScheduledExecutor() | ||
| timeoutExecutor.schedule(new Runnable { | ||
| override def run(): Unit = { | ||
| try { | ||
| timeoutCancel() | ||
| } catch { | ||
| case NonFatal(e) => | ||
| setOperationException(new HiveSQLException(e)) | ||
| logError(s"Error cancelling the query after timeout: $timeout seconds") | ||
| } finally { | ||
| timeoutExecutor.shutdown() | ||
| } | ||
| } | ||
| }, timeout, TimeUnit.SECONDS) | ||
| } | ||
|
|
||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we need to call shutdown for this executor ? |
||
| if (!runInBackground) { | ||
| execute() | ||
| } else { | ||
|
|
@@ -328,6 +358,17 @@ private[hive] class SparkExecuteStatementOperation( | |
| } | ||
| } | ||
|
|
||
| def timeoutCancel(): Unit = { | ||
| synchronized { | ||
| if (!getStatus.getState.isTerminal) { | ||
| logInfo(s"Query with $statementId timed out after $timeout seconds") | ||
| setState(OperationState.TIMEDOUT) | ||
| cleanup() | ||
| HiveThriftServer2.eventManager.onStatementTimeout(statementId) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def cancel(): Unit = { | ||
| synchronized { | ||
| if (!getStatus.getState.isTerminal) { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you rewrite this as:
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated.