[KYUUBI #761] Reduce explicit hang for test
<!-- 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/NetEase/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?_ <!-- Please clarify why the changes are needed. For instance, 1. If you add a feature, you can talk about the use case of it. 2. If you fix a bug, you can clarify why it is a bug. --> 1. reduce one server bootstrap 2. remove explicitly hanging a test for 1 min long to wait for an unnecessary message ### _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.readthedocs.io/en/latest/tools/testing.html#running-tests) locally before make a pull request Closes #761 from yaooqinn/test2. Closes #761 1d5c8cb4 [Kent Yao] style e877348a [Kent Yao] simpler 7ade8ead [Kent Yao] simpler 46a11b8c [Kent Yao] Reduce explicit hang for test Authored-by: Kent Yao <yao@apache.org> Signed-off-by: Kent Yao <yao@apache.org>
This commit is contained in:
parent
9b4f0261fb
commit
ccce2d2792
@ -34,13 +34,29 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
|||||||
protected val user: String = Utils.currentUser
|
protected val user: String = Utils.currentUser
|
||||||
protected val patterns = Seq("", "*", "%", null, ".*", "_*", "_%", ".%")
|
protected val patterns = Seq("", "*", "%", null, ".*", "_*", "_%", ".%")
|
||||||
protected def jdbcUrl: String
|
protected def jdbcUrl: String
|
||||||
protected def sessionConfigs: Map[String, String] = Map.empty
|
private var _sessionConfs: Map[String, String] = Map.empty
|
||||||
|
private var _sparkHiveConfs: Map[String, String] = Map.empty
|
||||||
|
private var _sparkHiveVars: Map[String, String] = Map.empty
|
||||||
|
protected def sessionConfigs: Map[String, String] = _sessionConfs
|
||||||
protected def sparkHiveConfigs: Map[String, String] = {
|
protected def sparkHiveConfigs: Map[String, String] = {
|
||||||
// TODO: KYUUBI-504: forbid setting FRONTEND_BIND_HOST by connection string in engine side
|
// TODO: KYUUBI-504: forbid setting FRONTEND_BIND_HOST by connection string in engine side
|
||||||
Map(KyuubiConf.FRONTEND_BIND_HOST.key -> "localhost")
|
Map(KyuubiConf.FRONTEND_BIND_HOST.key -> "localhost") ++: _sparkHiveConfs
|
||||||
}
|
}
|
||||||
protected def sparkHiveVars: Map[String, String] = Map.empty
|
protected def sparkHiveVars: Map[String, String] = _sparkHiveVars
|
||||||
|
|
||||||
|
def withSessionConf[T](
|
||||||
|
sessionConfs: Map[String, String] = Map.empty)(
|
||||||
|
sparkHiveConfs: Map[String, String])(
|
||||||
|
sparkHiveVars: Map[String, String])(f: => T): T = {
|
||||||
|
this._sessionConfs = sessionConfs
|
||||||
|
this._sparkHiveConfs = sparkHiveConfs
|
||||||
|
this._sparkHiveVars = sparkHiveVars
|
||||||
|
try f finally {
|
||||||
|
_sparkHiveVars = Map.empty
|
||||||
|
_sparkHiveConfs = Map.empty
|
||||||
|
_sessionConfs = Map.empty
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private def jdbcUrlWithConf: String = {
|
private def jdbcUrlWithConf: String = {
|
||||||
val sessionConfStr = sessionConfigs.map(kv => kv._1 + "=" + kv._2).mkString(";")
|
val sessionConfStr = sessionConfigs.map(kv => kv._1 + "=" + kv._2).mkString(";")
|
||||||
|
|||||||
@ -160,7 +160,7 @@ private[kyuubi] class EngineRef private(conf: KyuubiConf, user: String, sessionI
|
|||||||
process.destroyForcibly()
|
process.destroyForcibly()
|
||||||
MetricsSystem.tracing(_.incCount(MetricRegistry.name(ENGINE_TIMEOUT, appUser)))
|
MetricsSystem.tracing(_.incCount(MetricRegistry.name(ENGINE_TIMEOUT, appUser)))
|
||||||
throw KyuubiSQLException(
|
throw KyuubiSQLException(
|
||||||
s"Timeout($timeout) to launched Spark with $builder",
|
s"Timeout($timeout ms) to launched Spark with $builder",
|
||||||
builder.getError)
|
builder.getError)
|
||||||
}
|
}
|
||||||
engineRef = get(zkClient)
|
engineRef = get(zkClient)
|
||||||
|
|||||||
@ -1,67 +0,0 @@
|
|||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
|
||||||
* contributor license agreements. See the NOTICE file distributed with
|
|
||||||
* this work for additional information regarding copyright ownership.
|
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
|
||||||
* (the "License"); you may not use this file except in compliance with
|
|
||||||
* the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.kyuubi.engine
|
|
||||||
|
|
||||||
import java.util.concurrent.CountDownLatch
|
|
||||||
|
|
||||||
import org.apache.kyuubi.{KyuubiSQLException, WithKyuubiServerOnYarn}
|
|
||||||
import org.apache.kyuubi.config.KyuubiConf
|
|
||||||
import org.apache.kyuubi.operation.JDBCTestUtils
|
|
||||||
|
|
||||||
class EngineSubmitSuite extends WithKyuubiServerOnYarn with JDBCTestUtils {
|
|
||||||
override protected def jdbcUrl: String = getJdbcUrl
|
|
||||||
override protected val kyuubiServerConf: KyuubiConf = KyuubiConf()
|
|
||||||
override protected val connectionConf: Map[String, String] = {
|
|
||||||
Map("spark.yarn.queue" -> "two_cores_queue",
|
|
||||||
"spark.master" -> "yarn",
|
|
||||||
"spark.submit.deployMode" -> "client",
|
|
||||||
"spark.executor.instances" -> "1",
|
|
||||||
"spark.driver.cores" -> "1",
|
|
||||||
"spark.executor.cores" -> "1",
|
|
||||||
KyuubiConf.ENGINE_SHARE_LEVEL.key -> "connection",
|
|
||||||
KyuubiConf.ENGINE_INIT_TIMEOUT.key -> "60000")
|
|
||||||
}
|
|
||||||
|
|
||||||
test("submit spark app timeout with accepted status") {
|
|
||||||
@volatile var appIsRunning = false
|
|
||||||
val lock = new CountDownLatch(1)
|
|
||||||
new Thread(() => {
|
|
||||||
while (!appIsRunning) { Thread.sleep(100) }
|
|
||||||
try {
|
|
||||||
withJdbcStatement() { statement =>
|
|
||||||
val exception = intercept[KyuubiSQLException] {
|
|
||||||
statement.execute("select 1")
|
|
||||||
}
|
|
||||||
|
|
||||||
assert(exception.getMessage.contains("Failed to detect the root cause"))
|
|
||||||
assert(exception.getMessage.contains("The last line log"))
|
|
||||||
assert(exception.getMessage.contains("state: ACCEPTED"))
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
lock.countDown()
|
|
||||||
}
|
|
||||||
}).start()
|
|
||||||
|
|
||||||
withJdbcStatement() { statement =>
|
|
||||||
appIsRunning = true
|
|
||||||
statement.execute("select 1")
|
|
||||||
// hold resource so that the queue has no resource for other app
|
|
||||||
lock.await()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
@ -29,24 +29,6 @@ class SparkSqlEngineSuite extends WithKyuubiServer with JDBCTestUtils {
|
|||||||
.set(SESSION_CONF_RESTRICT_LIST.key, "kyuubi.xyz.abc,spark.sql.xyz.abc,spark.sql.xyz.abc.var")
|
.set(SESSION_CONF_RESTRICT_LIST.key, "kyuubi.xyz.abc,spark.sql.xyz.abc,spark.sql.xyz.abc.var")
|
||||||
}
|
}
|
||||||
|
|
||||||
private var _sessionConfs: Map[String, String] = Map.empty
|
|
||||||
private var _sparkHiveConfs: Map[String, String] = Map.empty
|
|
||||||
private var _sparkHiveVars: Map[String, String] = Map.empty
|
|
||||||
|
|
||||||
private def withSessionConf[T](
|
|
||||||
sessionConfs: Map[String, String])(
|
|
||||||
sparkHiveConfs: Map[String, String])(
|
|
||||||
sparkHiveVars: Map[String, String])(f: => T): T = {
|
|
||||||
this._sessionConfs = sessionConfs
|
|
||||||
this._sparkHiveConfs = sparkHiveConfs
|
|
||||||
this._sparkHiveVars = sparkHiveVars
|
|
||||||
try f finally {
|
|
||||||
_sparkHiveVars = Map.empty
|
|
||||||
_sparkHiveConfs = Map.empty
|
|
||||||
_sessionConfs = Map.empty
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
test("ignore config via system settings") {
|
test("ignore config via system settings") {
|
||||||
val sessionConf = Map("kyuubi.abc.xyz" -> "123", "kyuubi.abc.xyz0" -> "123")
|
val sessionConf = Map("kyuubi.abc.xyz" -> "123", "kyuubi.abc.xyz0" -> "123")
|
||||||
val sparkHiveConfs = Map("spark.sql.abc.xyz" -> "123", "spark.sql.abc.xyz0" -> "123")
|
val sparkHiveConfs = Map("spark.sql.abc.xyz" -> "123", "spark.sql.abc.xyz0" -> "123")
|
||||||
@ -143,16 +125,4 @@ class SparkSqlEngineSuite extends WithKyuubiServer with JDBCTestUtils {
|
|||||||
}
|
}
|
||||||
|
|
||||||
override protected def jdbcUrl: String = getJdbcUrl
|
override protected def jdbcUrl: String = getJdbcUrl
|
||||||
|
|
||||||
override protected def sessionConfigs: Map[String, String] = {
|
|
||||||
super.sessionConfigs ++: _sessionConfs
|
|
||||||
}
|
|
||||||
|
|
||||||
override protected def sparkHiveConfigs: Map[String, String] = {
|
|
||||||
super.sparkHiveConfigs ++: _sparkHiveConfs
|
|
||||||
}
|
|
||||||
|
|
||||||
override protected def sparkHiveVars: Map[String, String] = {
|
|
||||||
super.sparkHiveVars ++: _sparkHiveVars
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@ -17,10 +17,12 @@
|
|||||||
|
|
||||||
package org.apache.kyuubi.operation
|
package org.apache.kyuubi.operation
|
||||||
|
|
||||||
|
import java.sql.SQLException
|
||||||
|
|
||||||
import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetOperationStatusReq, TOperationState, TStatusCode}
|
import org.apache.hive.service.rpc.thrift.{TExecuteStatementReq, TGetOperationStatusReq, TOperationState, TStatusCode}
|
||||||
import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
|
import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
|
||||||
|
|
||||||
import org.apache.kyuubi.WithKyuubiServer
|
import org.apache.kyuubi.{KyuubiSQLException, WithKyuubiServer}
|
||||||
import org.apache.kyuubi.config.KyuubiConf
|
import org.apache.kyuubi.config.KyuubiConf
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -51,4 +53,16 @@ class KyuubiOperationPerConnectionSuite extends WithKyuubiServer with JDBCTestUt
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
test("submit spark app timeout with last log output") {
|
||||||
|
withSessionConf()(Map(KyuubiConf.ENGINE_INIT_TIMEOUT.key -> "2000"))(Map.empty) {
|
||||||
|
val exception = intercept[SQLException] {
|
||||||
|
withJdbcStatement() { statement => // no-op
|
||||||
|
}
|
||||||
|
}
|
||||||
|
val verboseMessage = KyuubiSQLException.stringifyException(exception)
|
||||||
|
assert(verboseMessage.contains("Failed to detect the root cause"))
|
||||||
|
assert(verboseMessage.contains("The last line log"))
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
Loading…
Reference in New Issue
Block a user