Skip to content

Commit 34043ae

Browse files
SteNicholasyanghua
authored andcommitted
[KYUUBI #1829] Support executing statement async for Flink SQL engine
<!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html 2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'. 3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'. --> ### _Why are the changes needed?_ Support executing statement async for Flink SQL engine. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1845 from SteNicholas/KYUUBI-1829. Closes #1829 956739c [SteNicholas] [KYUUBI #1829] Support executing statement async for Flink SQL engine 4ccf4b4 [SteNicholas] [KYUUBI #1829] Support executing statement async for Flink SQL engine 616ddd2 [SteNicholas] [KYUUBI #1829] Support executing statement async for Flink SQL engine Authored-by: SteNicholas <programgeek@163.com> Signed-off-by: yanghua <yanghua1127@gmail.com>
1 parent cebc03a commit 34043ae

File tree

4 files changed

+31
-8
lines changed

4 files changed

+31
-8
lines changed

externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import scala.collection.JavaConverters._
2424
import scala.collection.mutable.ArrayBuffer
2525

2626
import com.google.common.annotations.VisibleForTesting
27+
import org.apache.calcite.rel.metadata.{DefaultRelMetadataProvider, JaninoRelMetadataProvider, RelMetadataQueryBase}
2728
import org.apache.flink.table.api.{DataTypes, ResultKind}
2829
import org.apache.flink.table.catalog.Column
2930
import org.apache.flink.table.client.gateway.{Executor, TypedResult}
@@ -75,11 +76,11 @@ class ExecuteStatement(
7576
val asyncOperation = new Runnable {
7677
override def run(): Unit = {
7778
OperationLog.setCurrentOperationLog(operationLog)
79+
executeStatement()
7880
}
7981
}
8082

8183
try {
82-
executeStatement()
8384
val flinkSQLSessionManager = session.sessionManager
8485
val backgroundHandle = flinkSQLSessionManager.submitBackgroundOperation(asyncOperation)
8586
setBackgroundHandle(backgroundHandle)
@@ -100,6 +101,9 @@ class ExecuteStatement(
100101
try {
101102
setState(OperationState.RUNNING)
102103

104+
// set the thread variable THREAD_PROVIDERS
105+
RelMetadataQueryBase.THREAD_PROVIDERS.set(
106+
JaninoRelMetadataProvider.of(DefaultRelMetadataProvider.INSTANCE))
103107
val operation = executor.parseStatement(sessionId, statement)
104108
operation match {
105109
case queryOperation: QueryOperation => runQueryOperation(queryOperation)

externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperation.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -114,9 +114,7 @@ abstract class FlinkOperation(
114114
protected def cleanup(targetState: OperationState): Unit = state.synchronized {
115115
if (!isTerminalState(state)) {
116116
setState(targetState)
117-
if (shouldRunAsync) {
118-
Option(getBackgroundHandle).foreach(_.cancel(true))
119-
}
117+
Option(getBackgroundHandle).foreach(_.cancel(true))
120118
}
121119
}
122120

externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala

Lines changed: 25 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.sql.DatabaseMetaData
2222
import org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG
2323
import org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE
2424
import org.apache.flink.table.types.logical.LogicalTypeRoot
25+
import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TFetchResultsReq, TOpenSessionReq}
2526
import org.scalatest.concurrent.PatienceConfiguration.Timeout
2627
import org.scalatest.time.SpanSugar._
2728

@@ -35,7 +36,7 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
3536
override def withKyuubiConf: Map[String, String] = Map()
3637

3738
override protected def jdbcUrl: String =
38-
s"jdbc:hive2://${engine.frontendServices.head.connectionUrl}/"
39+
s"jdbc:hive2://${engine.frontendServices.head.connectionUrl}/;"
3940

4041
ignore("release session if shared level is CONNECTION") {
4142
logger.info(s"jdbc url is $jdbcUrl")
@@ -734,4 +735,27 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
734735
assert(resultSet.getString(1) === "a")
735736
}
736737
}
738+
739+
test("async execute statement - select column name with dots") {
740+
withThriftClient { client =>
741+
val tOpenSessionReq = new TOpenSessionReq()
742+
tOpenSessionReq.setUsername("kentyao")
743+
tOpenSessionReq.setPassword("anonymous")
744+
val tOpenSessionResp = client.OpenSession(tOpenSessionReq)
745+
val tExecuteStatementReq = new TExecuteStatementReq()
746+
tExecuteStatementReq.setSessionHandle(tOpenSessionResp.getSessionHandle)
747+
tExecuteStatementReq.setRunAsync(true)
748+
tExecuteStatementReq.setStatement("select 'tmp.hello'")
749+
val tExecuteStatementResp = client.ExecuteStatement(tExecuteStatementReq)
750+
val operationHandle = tExecuteStatementResp.getOperationHandle
751+
waitForOperationToComplete(client, operationHandle)
752+
val tFetchResultsReq = new TFetchResultsReq()
753+
tFetchResultsReq.setOperationHandle(operationHandle)
754+
tFetchResultsReq.setFetchType(2)
755+
tFetchResultsReq.setMaxRows(1000)
756+
val tFetchResultsResp = client.FetchResults(tFetchResultsReq)
757+
assert(tFetchResultsResp.getResults.getColumns.get(0)
758+
.getStringVal.getValues.get(0) === "tmp.hello")
759+
}
760+
}
737761
}

externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkOperationSuite.scala

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -323,9 +323,6 @@ class SparkOperationSuite extends WithSparkSQLEngine with HiveMetadataTests with
323323

324324
test("basic open | execute | close") {
325325
withThriftClient { client =>
326-
val operationManager =
327-
engine.backendService.sessionManager.operationManager.asInstanceOf[SparkSQLOperationManager]
328-
329326
val req = new TOpenSessionReq()
330327
req.setUsername("kentyao")
331328
req.setPassword("anonymous")

0 commit comments

Comments
 (0)