diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala index 7e4f31f8a..8838799bc 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala @@ -19,12 +19,14 @@ package org.apache.kyuubi.engine.flink import java.io.File import java.nio.file.Paths +import java.time.Duration import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ import org.apache.flink.configuration.{Configuration, DeploymentOptions, GlobalConfiguration, PipelineOptions} import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.gateway.api.config.SqlGatewayServiceConfigOptions import org.apache.flink.table.gateway.service.context.DefaultContext import org.apache.kyuubi.{Logging, Utils} @@ -146,6 +148,11 @@ object FlinkSQLEngine extends Logging { kyuubiConf.getOption(KYUUBI_SESSION_USER_KEY) .foreach(flinkConf.setString(KYUUBI_SESSION_USER_KEY, _)) + // force disable Flink's session timeout + flinkConf.set( + SqlGatewayServiceConfigOptions.SQL_GATEWAY_SESSION_IDLE_TIMEOUT, + Duration.ofMillis(0)) + executionTarget match { case "yarn-per-job" | "yarn-application" => if (flinkConf.containsKey("high-availability.cluster-id")) {