Compare commits
14 Commits
master
...
branch-1.1
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
9355c999d3 | ||
|
|
b15b42cadf | ||
|
|
12fbef030d | ||
|
|
14b2d82630 | ||
|
|
a00b9fa9d0 | ||
|
|
1503af361b | ||
|
|
68cf158279 | ||
|
|
b81f53d6c2 | ||
|
|
6afacdcdb8 | ||
|
|
46725901c6 | ||
|
|
5e551bdd71 | ||
|
|
7107666f1b | ||
|
|
48e359b4a7 | ||
|
|
49094f7768 |
25
.github/workflows/release.yml
vendored
25
.github/workflows/release.yml
vendored
@ -9,14 +9,6 @@ jobs:
|
||||
build:
|
||||
name: Create and Publish Release
|
||||
runs-on: ubuntu-latest
|
||||
strategy:
|
||||
matrix:
|
||||
profiles:
|
||||
- '-Pspark-3.0 -Pspark-hadoop-2.7'
|
||||
- '-Pspark-3.0 -Pspark-hadoop-3.2'
|
||||
- '-Pspark-3.1 -Pspark-hadoop-2.7'
|
||||
- '-Pspark-3.1 -Pspark-hadoop-3.2'
|
||||
- '--spark-provided'
|
||||
steps:
|
||||
- uses: actions/checkout@master
|
||||
# We split caches because GitHub Action Cache has a 400MB-size limit.
|
||||
@ -42,8 +34,16 @@ jobs:
|
||||
uses: actions/setup-java@v1
|
||||
with:
|
||||
java-version: '1.8'
|
||||
- name: Make Distribution
|
||||
run: ./build/dist --tgz ${{ matrix.profiles }}
|
||||
- name: Make Distribution -Pspark-3.0
|
||||
run: ./build/dist --tgz -Pspark-3.0
|
||||
- name: Make Distribution -Pspark-3.0 -Pspark-hadoop-3.2
|
||||
run: ./build/dist --tgz -Pspark-3.0 -Pspark-hadoop-3.2
|
||||
- name: Make Distribution -Pspark-3.1
|
||||
run: ./build/dist --tgz -Pspark-3.1
|
||||
- name: Make Distribution -Pspark-3.1 -Pspark-hadoop-3.2
|
||||
run: ./build/dist --tgz -Pspark-3.1 -Pspark-hadoop-3.2
|
||||
- name: Make Distribution --spark-provided
|
||||
run: ./build/dist --tgz --spark-provided
|
||||
- name: Create Release
|
||||
id: create_release
|
||||
uses: actions/create-release@v1
|
||||
@ -60,3 +60,8 @@ jobs:
|
||||
GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
|
||||
with:
|
||||
asset_paths: '["./kyuubi-*tar.gz"]'
|
||||
- name: Update Release Note
|
||||
id: github_release_changelog
|
||||
uses: mikepenz/release-changelog-builder-action@v1
|
||||
env:
|
||||
GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
|
||||
|
||||
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
@ -32,6 +32,10 @@
|
||||
<name>Kyuubi Project Dev TPCDS Generator</name>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_${scala.binary.version}</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_${scala.binary.version}</artifactId>
|
||||
|
||||
@ -52,7 +52,6 @@ case class TableGenerator(
|
||||
|
||||
private def toDF: DataFrame = {
|
||||
val rawRDD = ss.sparkContext.parallelize(1 to parallelism, parallelism).flatMap { i =>
|
||||
val parallel = s"-PARALLEL $parallelism -child $i"
|
||||
val os = System.getProperty("os.name").split(' ')(0).toLowerCase
|
||||
val loader = Thread.currentThread().getContextClassLoader
|
||||
|
||||
@ -85,20 +84,21 @@ case class TableGenerator(
|
||||
val cmd = s"./dsdgen" +
|
||||
s" -TABLE $name" +
|
||||
s" -SCALE $scaleFactor" +
|
||||
s" $parallel" +
|
||||
s" -PARALLEL $parallelism" +
|
||||
s" -child $i" +
|
||||
s" -DISTRIBUTIONS tpcds.idx" +
|
||||
s" -FORCE Y" +
|
||||
s" -QUIET Y"
|
||||
|
||||
val builder = new ProcessBuilder(cmd.split(" "): _*)
|
||||
builder.directory(tempDir)
|
||||
logger.info(s"Start $cmd at " + builder.directory())
|
||||
builder.redirectError(Redirect.INHERIT)
|
||||
val data = Paths.get(tempDir.toString, s"${name}_${i}_$parallelism.dat")
|
||||
logger.info(s"Start $cmd at ${builder.directory()}")
|
||||
val process = builder.start()
|
||||
val res = process.waitFor()
|
||||
|
||||
logger.info(s"Finish w/ $res" + cmd)
|
||||
logger.info(s"Finish w/ $res ${cmd}")
|
||||
val data = Paths.get(tempDir.toString, s"${name}_${i}_$parallelism.dat")
|
||||
val iterator = if (Files.exists(data)) {
|
||||
new BufferedSource(new FileInputStream(data.toFile), 8192).getLines()
|
||||
} else {
|
||||
|
||||
@ -285,6 +285,9 @@ kyuubi\.kinit<br>\.principal|<div style='width: 80pt;word-wrap: break-word;white
|
||||
Key | Default | Meaning | Since
|
||||
--- | --- | --- | ---
|
||||
kyuubi\.operation\.idle<br>\.timeout|<div style='width: 80pt;word-wrap: break-word;white-space: normal'>PT3H</div>|<div style='width: 200pt;word-wrap: break-word;white-space: normal'>Operation will be closed when it's not accessed for this duration of time</div>|<div style='width: 20pt'>1.0.0</div>
|
||||
kyuubi\.operation<br>\.interrupt\.on\.cancel|<div style='width: 80pt;word-wrap: break-word;white-space: normal'>true</div>|<div style='width: 200pt;word-wrap: break-word;white-space: normal'>When true, all running tasks will be interrupted if one cancels a query. When false, all running tasks will remain until finished.</div>|<div style='width: 20pt'>1.2.0</div>
|
||||
kyuubi\.operation<br>\.query\.timeout|<div style='width: 80pt;word-wrap: break-word;white-space: normal'>PT0S</div>|<div style='width: 200pt;word-wrap: break-word;white-space: normal'>Set a query duration timeout in seconds in Kyuubi. If the timeout is set to a positive value, a running query will be cancelled automatically if timeout. 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. If you set this timeout and prefer to cancel the queries right away without waiting task to finish, consider enabling kyuubi.operation.interrupt.on.cancel together.</div>|<div style='width: 20pt'>1.2.0</div>
|
||||
kyuubi\.operation<br>\.scheduler\.pool|<div style='width: 80pt;word-wrap: break-word;white-space: normal'><undefined></div>|<div style='width: 200pt;word-wrap: break-word;white-space: normal'>The scheduler pool of job. Note that, this config should be used after change Spark config spark.scheduler.mode=FAIR.</div>|<div style='width: 20pt'>1.1.1</div>
|
||||
kyuubi\.operation<br>\.status\.polling<br>\.timeout|<div style='width: 80pt;word-wrap: break-word;white-space: normal'>PT5S</div>|<div style='width: 200pt;word-wrap: break-word;white-space: normal'>Timeout(ms) for long polling asynchronous running sql query's status</div>|<div style='width: 20pt'>1.0.0</div>
|
||||
|
||||
### Session
|
||||
|
||||
@ -12,7 +12,7 @@ Welcome to Kyuubi's documentation!
|
||||
|
||||
Kyuubi™ is a unified multi-tenant JDBC interface for large-scale data processing and analytics, built on top of `Apache Spark™ <http://spark.apache.org/>`_.
|
||||
|
||||
.. image:: ./imgs/kyuubi_layers.*
|
||||
.. image:: ./imgs/kyuubi_layers.png
|
||||
|
||||
In general, the complete ecosystem of Kyuubi falls into the hierarchies shown in the above figure, with each layer loosely coupled to the other.
|
||||
|
||||
|
||||
2
externals/kyuubi-download/pom.xml
vendored
2
externals/kyuubi-download/pom.xml
vendored
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
2
externals/kyuubi-spark-sql-engine/pom.xml
vendored
2
externals/kyuubi-spark-sql-engine/pom.xml
vendored
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
@ -17,24 +17,34 @@
|
||||
|
||||
package org.apache.kyuubi.engine.spark.operation
|
||||
|
||||
import java.util.concurrent.RejectedExecutionException
|
||||
import java.util.concurrent.{RejectedExecutionException, TimeUnit}
|
||||
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
import org.apache.kyuubi.{KyuubiSQLException, Logging}
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.engine.spark.{ArrayFetchIterator, KyuubiSparkUtil}
|
||||
import org.apache.kyuubi.operation.{OperationState, OperationType}
|
||||
import org.apache.kyuubi.operation.log.OperationLog
|
||||
import org.apache.kyuubi.session.Session
|
||||
import org.apache.kyuubi.util.ThreadUtils
|
||||
|
||||
class ExecuteStatement(
|
||||
spark: SparkSession,
|
||||
session: Session,
|
||||
protected override val statement: String,
|
||||
override val shouldRunAsync: Boolean)
|
||||
override val shouldRunAsync: Boolean,
|
||||
queryTimeout: Long)
|
||||
extends SparkOperation(spark, OperationType.EXECUTE_STATEMENT, session) with Logging {
|
||||
|
||||
private val forceCancel =
|
||||
session.sessionManager.getConf.get(KyuubiConf.OPERATION_FORCE_CANCEL)
|
||||
|
||||
private val schedulerPool =
|
||||
spark.conf.getOption(KyuubiConf.OPERATION_SCHEDULER_POOL.key).orElse(
|
||||
session.sessionManager.getConf.get(KyuubiConf.OPERATION_SCHEDULER_POOL))
|
||||
|
||||
private val operationLog: OperationLog =
|
||||
OperationLog.createOperationLog(session.handle, getHandle)
|
||||
override def getOperationLog: Option[OperationLog] = Option(operationLog)
|
||||
@ -58,24 +68,22 @@ class ExecuteStatement(
|
||||
OperationLog.removeCurrentOperationLog()
|
||||
}
|
||||
|
||||
private def executeStatement(): Unit = {
|
||||
private def executeStatement(): Unit = withLocalProperties {
|
||||
try {
|
||||
setState(OperationState.RUNNING)
|
||||
info(KyuubiSparkUtil.diagnostics(spark))
|
||||
Thread.currentThread().setContextClassLoader(spark.sharedState.jarClassLoader)
|
||||
spark.sparkContext.setJobGroup(statementId, statement)
|
||||
result = spark.sql(statement)
|
||||
debug(result.queryExecution)
|
||||
iter = new ArrayFetchIterator(result.collect())
|
||||
setState(OperationState.FINISHED)
|
||||
} catch {
|
||||
onError(cancel = true)
|
||||
} finally {
|
||||
spark.sparkContext.clearJobGroup()
|
||||
}
|
||||
}
|
||||
|
||||
override protected def runInternal(): Unit = {
|
||||
addTimeoutMonitor()
|
||||
if (shouldRunAsync) {
|
||||
val asyncOperation = new Runnable {
|
||||
override def run(): Unit = {
|
||||
@ -100,4 +108,33 @@ class ExecuteStatement(
|
||||
executeStatement()
|
||||
}
|
||||
}
|
||||
|
||||
private def withLocalProperties[T](f: => T): T = {
|
||||
try {
|
||||
spark.sparkContext.setJobGroup(statementId, statement, forceCancel)
|
||||
schedulerPool match {
|
||||
case Some(pool) =>
|
||||
spark.sparkContext.setLocalProperty("spark.scheduler.pool", pool)
|
||||
case None =>
|
||||
}
|
||||
|
||||
f
|
||||
} finally {
|
||||
spark.sparkContext.setLocalProperty("spark.scheduler.pool", null)
|
||||
spark.sparkContext.clearJobGroup()
|
||||
}
|
||||
}
|
||||
|
||||
private def addTimeoutMonitor(): Unit = {
|
||||
if (queryTimeout > 0) {
|
||||
val timeoutExecutor =
|
||||
ThreadUtils.newDaemonSingleThreadScheduledExecutor("query-timeout-thread")
|
||||
timeoutExecutor.schedule(new Runnable {
|
||||
override def run(): Unit = {
|
||||
cleanup(OperationState.TIMEOUT)
|
||||
timeoutExecutor.shutdown()
|
||||
}
|
||||
}, queryTimeout, TimeUnit.SECONDS)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -49,9 +49,10 @@ abstract class SparkOperation(spark: SparkSession, opType: OperationType, sessio
|
||||
|
||||
protected def resultSchema: StructType
|
||||
|
||||
protected def cleanup(targetState: OperationState): Unit = synchronized {
|
||||
protected def cleanup(targetState: OperationState): Unit = state.synchronized {
|
||||
if (!isTerminalState(state)) {
|
||||
setState(targetState)
|
||||
Option(getBackgroundHandle).foreach(_.cancel(true))
|
||||
spark.sparkContext.cancelJobGroup(statementId)
|
||||
}
|
||||
}
|
||||
@ -92,7 +93,11 @@ abstract class SparkOperation(spark: SparkSession, opType: OperationType, sessio
|
||||
if (cancel) spark.sparkContext.cancelJobGroup(statementId)
|
||||
state.synchronized {
|
||||
val errMsg = KyuubiSQLException.stringifyException(e)
|
||||
if (isTerminalState(state)) {
|
||||
if (state == OperationState.TIMEOUT) {
|
||||
val ke = KyuubiSQLException(s"Timeout operating $opType: $errMsg")
|
||||
setOperationException(ke)
|
||||
throw ke
|
||||
} else if (isTerminalState(state)) {
|
||||
warn(s"Ignore exception in terminal state with $statementId: $errMsg")
|
||||
} else {
|
||||
setState(OperationState.ERROR)
|
||||
|
||||
@ -58,7 +58,7 @@ class SparkSQLOperationManager private (name: String) extends OperationManager(n
|
||||
runAsync: Boolean,
|
||||
queryTimeout: Long): Operation = {
|
||||
val spark = getSparkSession(session.handle)
|
||||
val operation = new ExecuteStatement(spark, session, statement, runAsync)
|
||||
val operation = new ExecuteStatement(spark, session, statement, runAsync, queryTimeout)
|
||||
addOperation(operation)
|
||||
}
|
||||
|
||||
|
||||
30
externals/kyuubi-spark-sql-engine/src/test/resources/test-scheduler-pool.xml
vendored
Normal file
30
externals/kyuubi-spark-sql-engine/src/test/resources/test-scheduler-pool.xml
vendored
Normal file
@ -0,0 +1,30 @@
|
||||
<?xml version="1.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.
|
||||
-->
|
||||
|
||||
<allocations>
|
||||
<pool name="p0">
|
||||
<minShare>2</minShare>
|
||||
<weight>1</weight>
|
||||
<schedulingMode>FAIR</schedulingMode>
|
||||
</pool>
|
||||
<pool name="p1">
|
||||
<minShare>0</minShare>
|
||||
<weight>1</weight>
|
||||
<schedulingMode>FAIR</schedulingMode>
|
||||
</pool>
|
||||
</allocations>
|
||||
@ -0,0 +1,91 @@
|
||||
/*
|
||||
* 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.spark
|
||||
|
||||
import java.sql.{SQLTimeoutException, Statement}
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
|
||||
|
||||
import org.apache.spark.TaskKilled
|
||||
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
import org.scalatest.time.SpanSugar._
|
||||
|
||||
import org.apache.kyuubi.KyuubiFunSuite
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.operation.JDBCTestUtils
|
||||
|
||||
class SparkEngineSuites extends KyuubiFunSuite {
|
||||
|
||||
test("Add config to control if cancel invoke interrupt task on engine") {
|
||||
Seq(true, false).foreach { force =>
|
||||
withSparkJdbcStatement(Map(KyuubiConf.OPERATION_FORCE_CANCEL.key -> force.toString)) {
|
||||
case (statement, spark) =>
|
||||
val index = new AtomicInteger(0)
|
||||
val forceCancel = new AtomicBoolean(false)
|
||||
val listener = new SparkListener {
|
||||
override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
|
||||
assert(taskEnd.reason.isInstanceOf[TaskKilled])
|
||||
if (forceCancel.get()) {
|
||||
assert(System.currentTimeMillis() - taskEnd.taskInfo.launchTime < 3000)
|
||||
index.incrementAndGet()
|
||||
} else {
|
||||
assert(System.currentTimeMillis() - taskEnd.taskInfo.launchTime >= 4000)
|
||||
index.incrementAndGet()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
spark.sparkContext.addSparkListener(listener)
|
||||
try {
|
||||
statement.setQueryTimeout(3)
|
||||
forceCancel.set(force)
|
||||
val e1 = intercept[SQLTimeoutException] {
|
||||
statement.execute("select java_method('java.lang.Thread', 'sleep', 5000L)")
|
||||
}.getMessage
|
||||
assert(e1.contains("Query timed out"))
|
||||
eventually(Timeout(30.seconds)) {
|
||||
assert(index.get() == 1)
|
||||
}
|
||||
} finally {
|
||||
spark.sparkContext.removeSparkListener(listener)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def withSparkJdbcStatement(
|
||||
conf: Map[String, String] = Map.empty)(
|
||||
statement: (Statement, SparkSession) => Unit): Unit = {
|
||||
val spark = new WithSparkSuite {
|
||||
override def withKyuubiConf: Map[String, String] = conf
|
||||
override protected def jdbcUrl: String = getJdbcUrl
|
||||
}
|
||||
spark.startSparkEngine()
|
||||
val tmp: Statement => Unit = { tmpStatement =>
|
||||
statement(tmpStatement, spark.getSpark)
|
||||
}
|
||||
try {
|
||||
spark.withJdbcStatement()(tmp)
|
||||
} finally {
|
||||
spark.stopSparkEngine()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
trait WithSparkSuite extends WithSparkSQLEngine with JDBCTestUtils
|
||||
@ -0,0 +1,113 @@
|
||||
/*
|
||||
* 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.spark
|
||||
|
||||
import java.util.concurrent.Executors
|
||||
|
||||
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd, SparkListenerJobStart}
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
|
||||
|
||||
import org.apache.kyuubi.operation.JDBCTestUtils
|
||||
|
||||
class SchedulerPoolSuite extends WithSparkSQLEngine with JDBCTestUtils {
|
||||
override protected def jdbcUrl: String = getJdbcUrl
|
||||
override def withKyuubiConf: Map[String, String] = {
|
||||
val poolFile =
|
||||
Thread.currentThread().getContextClassLoader.getResource("test-scheduler-pool.xml")
|
||||
Map("spark.scheduler.mode" -> "FAIR",
|
||||
"spark.scheduler.allocation.file" -> poolFile.getFile,
|
||||
"spark.master" -> "local[2]")
|
||||
}
|
||||
|
||||
test("Scheudler pool") {
|
||||
@volatile var job0Started = false
|
||||
@volatile var job1StartTime = 0L
|
||||
@volatile var job2StartTime = 0L
|
||||
@volatile var job1FinishTime = 0L
|
||||
@volatile var job2FinishTime = 0L
|
||||
val listener = new SparkListener {
|
||||
override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
|
||||
info(jobStart)
|
||||
jobStart.jobId match {
|
||||
case 1 => job1StartTime = jobStart.time
|
||||
case 2 => job2StartTime = jobStart.time
|
||||
case 0 => job0Started = true
|
||||
}
|
||||
}
|
||||
|
||||
override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
|
||||
info(jobEnd)
|
||||
jobEnd.jobId match {
|
||||
case 1 => job1FinishTime = jobEnd.time
|
||||
case 2 => job2FinishTime = jobEnd.time
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
spark.sparkContext.addSparkListener(listener)
|
||||
|
||||
try {
|
||||
val threads = Executors.newFixedThreadPool(3)
|
||||
threads.execute(new Runnable {
|
||||
override def run(): Unit = {
|
||||
withJdbcStatement() { statement =>
|
||||
statement.execute("SET kyuubi.operation.scheduler.pool=p0")
|
||||
statement.execute("SELECT java_method('java.lang.Thread', 'sleep', 5000l)" +
|
||||
"FROM range(1, 3, 1, 2)")
|
||||
}
|
||||
}
|
||||
})
|
||||
// make sure job0 started then we have no resource right now
|
||||
eventually(Timeout(3.seconds)) {
|
||||
assert(job0Started)
|
||||
}
|
||||
Seq(1, 0).foreach { priority =>
|
||||
threads.execute(new Runnable {
|
||||
override def run(): Unit = {
|
||||
priority match {
|
||||
case 0 =>
|
||||
withJdbcStatement() { statement =>
|
||||
statement.execute("SET kyuubi.operation.scheduler.pool=p0")
|
||||
statement.execute("SELECT java_method('java.lang.Thread', 'sleep', 1500l)" +
|
||||
"FROM range(1, 3, 1, 2)")
|
||||
}
|
||||
|
||||
case 1 =>
|
||||
withJdbcStatement() { statement =>
|
||||
statement.execute("SET kyuubi.operation.scheduler.pool=p1")
|
||||
statement.execute("SELECT java_method('java.lang.Thread', 'sleep', 1500l)" +
|
||||
" FROM range(1, 3, 1, 2)")
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
threads.shutdown()
|
||||
eventually(Timeout(10.seconds)) {
|
||||
// We can not ensure that job1 is started before job2 so here using abs.
|
||||
assert(Math.abs(job1StartTime - job2StartTime) < 1000)
|
||||
// Job1 minShare is 2(total resource) so that job2 should be allocated tasks after
|
||||
// job1 finished.
|
||||
assert(job2FinishTime - job1FinishTime >= 1000)
|
||||
}
|
||||
} finally {
|
||||
spark.sparkContext.removeSparkListener(listener)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -36,7 +36,7 @@ trait WithSparkSQLEngine extends KyuubiFunSuite {
|
||||
super.beforeAll()
|
||||
}
|
||||
|
||||
protected def startSparkEngine(): Unit = {
|
||||
def startSparkEngine(): Unit = {
|
||||
val warehousePath = Utils.createTempDir()
|
||||
val metastorePath = Utils.createTempDir()
|
||||
warehousePath.toFile.delete()
|
||||
@ -63,7 +63,7 @@ trait WithSparkSQLEngine extends KyuubiFunSuite {
|
||||
stopSparkEngine()
|
||||
}
|
||||
|
||||
protected def stopSparkEngine(): Unit = {
|
||||
def stopSparkEngine(): Unit = {
|
||||
// we need to clean up conf since it's the global config in same jvm.
|
||||
withKyuubiConf.foreach { case (k, _) =>
|
||||
System.clearProperty(k)
|
||||
@ -83,4 +83,5 @@ trait WithSparkSQLEngine extends KyuubiFunSuite {
|
||||
}
|
||||
|
||||
protected def getJdbcUrl: String = s"jdbc:hive2://$connectionUrl/;"
|
||||
def getSpark: SparkSession = spark
|
||||
}
|
||||
|
||||
@ -24,7 +24,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
|
||||
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
@ -502,6 +502,27 @@ object KyuubiConf {
|
||||
.timeConf
|
||||
.createWithDefault(Duration.ofSeconds(5).toMillis)
|
||||
|
||||
val OPERATION_FORCE_CANCEL: ConfigEntry[Boolean] =
|
||||
buildConf("operation.interrupt.on.cancel")
|
||||
.doc("When true, all running tasks will be interrupted if one cancels a query. " +
|
||||
"When false, all running tasks will remain until finished.")
|
||||
.version("1.2.0")
|
||||
.booleanConf
|
||||
.createWithDefault(true)
|
||||
|
||||
val OPERATION_QUERY_TIMEOUT: ConfigEntry[Long] =
|
||||
buildConf("operation.query.timeout")
|
||||
.doc("Set a query duration timeout in seconds in Kyuubi. If the timeout is set to " +
|
||||
"a positive value, a running query will be cancelled automatically if timeout. " +
|
||||
"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. If you set this timeout and prefer " +
|
||||
"to cancel the queries right away without waiting task to finish, consider enabling " +
|
||||
s"${OPERATION_FORCE_CANCEL.key} together.")
|
||||
.version("1.2.0")
|
||||
.timeConf
|
||||
.createWithDefault(Duration.ZERO.toMillis)
|
||||
|
||||
val ENGINE_SHARED_LEVEL: ConfigEntry[String] = buildConf("session.engine.share.level")
|
||||
.doc("The SQL engine App will be shared in different levels, available configs are: <ul>" +
|
||||
" <li>CONNECTION: the App will not be shared but only used by the current client" +
|
||||
@ -514,4 +535,11 @@ object KyuubiConf {
|
||||
.transform(_.toUpperCase(Locale.ROOT))
|
||||
.checkValues(ShareLevel.values.map(_.toString))
|
||||
.createWithDefault(ShareLevel.USER.toString)
|
||||
|
||||
val OPERATION_SCHEDULER_POOL: OptionalConfigEntry[String] = buildConf("operation.scheduler.pool")
|
||||
.doc("The scheduler pool of job. Note that, this config should be used after change Spark " +
|
||||
"config spark.scheduler.mode=FAIR.")
|
||||
.version("1.1.1")
|
||||
.stringConf
|
||||
.createOptional
|
||||
}
|
||||
|
||||
@ -74,7 +74,7 @@ abstract class AbstractOperation(opType: OperationType, session: Session)
|
||||
var timeCost = ""
|
||||
newState match {
|
||||
case RUNNING => startTime = System.currentTimeMillis()
|
||||
case ERROR | FINISHED | CANCELED =>
|
||||
case ERROR | FINISHED | CANCELED | TIMEOUT =>
|
||||
completedTime = System.currentTimeMillis()
|
||||
timeCost = s", time taken: ${(completedTime - startTime) / 1000.0} seconds"
|
||||
case _ =>
|
||||
|
||||
@ -34,7 +34,7 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
||||
protected val patterns = Seq("", "*", "%", null, ".*", "_*", "_%", ".%")
|
||||
protected def jdbcUrl: String
|
||||
|
||||
protected def withMultipleConnectionJdbcStatement(
|
||||
def withMultipleConnectionJdbcStatement(
|
||||
tableNames: String*)(fs: (Statement => Unit)*): Unit = {
|
||||
val connections = fs.map { _ => DriverManager.getConnection(jdbcUrl, user, "") }
|
||||
val statements = connections.map(_.createStatement())
|
||||
@ -57,7 +57,7 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
protected def withDatabases(dbNames: String*)(fs: (Statement => Unit)*): Unit = {
|
||||
def withDatabases(dbNames: String*)(fs: (Statement => Unit)*): Unit = {
|
||||
val connections = fs.map { _ => DriverManager.getConnection(jdbcUrl, user, "") }
|
||||
val statements = connections.map(_.createStatement())
|
||||
|
||||
@ -75,11 +75,11 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
protected def withJdbcStatement(tableNames: String*)(f: Statement => Unit): Unit = {
|
||||
def withJdbcStatement(tableNames: String*)(f: Statement => Unit): Unit = {
|
||||
withMultipleConnectionJdbcStatement(tableNames: _*)(f)
|
||||
}
|
||||
|
||||
protected def withThriftClient(f: TCLIService.Iface => Unit): Unit = {
|
||||
def withThriftClient(f: TCLIService.Iface => Unit): Unit = {
|
||||
val hostAndPort = jdbcUrl.stripPrefix("jdbc:hive2://").split("/;").head.split(":")
|
||||
val host = hostAndPort.head
|
||||
val port = hostAndPort(1).toInt
|
||||
@ -96,7 +96,7 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
protected def withSessionHandle(f: (TCLIService.Iface, TSessionHandle) => Unit): Unit = {
|
||||
def withSessionHandle(f: (TCLIService.Iface, TSessionHandle) => Unit): Unit = {
|
||||
withThriftClient { client =>
|
||||
val req = new TOpenSessionReq()
|
||||
req.setUsername(user)
|
||||
@ -117,7 +117,7 @@ trait JDBCTestUtils extends KyuubiFunSuite {
|
||||
}
|
||||
}
|
||||
|
||||
protected def checkGetSchemas(
|
||||
def checkGetSchemas(
|
||||
rs: ResultSet, dbNames: Seq[String], catalogName: String = ""): Unit = {
|
||||
var count = 0
|
||||
while(rs.next()) {
|
||||
|
||||
@ -17,7 +17,7 @@
|
||||
|
||||
package org.apache.kyuubi.operation
|
||||
|
||||
import java.sql.{Date, SQLException, Timestamp}
|
||||
import java.sql.{Date, SQLException, SQLTimeoutException, Timestamp}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
@ -327,4 +327,26 @@ trait JDBCTests extends BasicJDBCTests {
|
||||
assert(metaData.getScale(1) === 0)
|
||||
}
|
||||
}
|
||||
|
||||
test("Support query auto timeout cancel on thriftserver - setQueryTimeout") {
|
||||
withJdbcStatement() { statement =>
|
||||
statement.setQueryTimeout(1)
|
||||
val e = intercept[SQLTimeoutException] {
|
||||
statement.execute("select java_method('java.lang.Thread', 'sleep', 10000L)")
|
||||
}.getMessage
|
||||
assert(e.contains("Query timed out after"))
|
||||
|
||||
statement.setQueryTimeout(0)
|
||||
val rs1 = statement.executeQuery(
|
||||
"select 'test', java_method('java.lang.Thread', 'sleep', 3000L)")
|
||||
rs1.next()
|
||||
assert(rs1.getString(1) == "test")
|
||||
|
||||
statement.setQueryTimeout(-1)
|
||||
val rs2 = statement.executeQuery(
|
||||
"select 'test', java_method('java.lang.Thread', 'sleep', 3000L)")
|
||||
rs2.next()
|
||||
assert(rs2.getString(1) == "test")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -37,6 +37,7 @@ class FrontendServiceSuite extends KyuubiFunSuite {
|
||||
|
||||
protected val server = new NoopServer()
|
||||
protected val conf = KyuubiConf()
|
||||
.set(KyuubiConf.FRONTEND_BIND_HOST, "127.0.0.1")
|
||||
.set(KyuubiConf.FRONTEND_BIND_PORT, 0)
|
||||
.set("kyuubi.test.server.should.fail", "false")
|
||||
|
||||
|
||||
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
@ -22,7 +22,7 @@
|
||||
<parent>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
|
||||
@ -17,7 +17,7 @@
|
||||
|
||||
package org.apache.kyuubi.engine
|
||||
|
||||
import java.io.{File, FilenameFilter, IOException}
|
||||
import java.io.{File, IOException}
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.nio.file.{Files, Path}
|
||||
|
||||
@ -65,15 +65,11 @@ trait ProcBuilder {
|
||||
// Visible for test
|
||||
private[kyuubi] var logCaptureThread: Thread = _
|
||||
|
||||
private lazy val engineLog: File = ProcBuilder.synchronized {
|
||||
private[kyuubi] lazy val engineLog: File = ProcBuilder.synchronized {
|
||||
val engineLogTimeout = conf.get(KyuubiConf.ENGINE_LOG_TIMEOUT)
|
||||
val currentTime = System.currentTimeMillis()
|
||||
val processLogPath = workingDir
|
||||
val totalExistsFile = processLogPath.toFile.listFiles(new FilenameFilter() {
|
||||
override def accept(dir: File, name: String): Boolean = {
|
||||
name.startsWith(module)
|
||||
}
|
||||
})
|
||||
val totalExistsFile = processLogPath.toFile.listFiles { (_, name) => name.startsWith(module) }
|
||||
val sorted = totalExistsFile.sortBy(_.getName.split("\\.").last.toInt)
|
||||
val nextIndex = if (sorted.isEmpty) {
|
||||
0
|
||||
@ -81,6 +77,18 @@ trait ProcBuilder {
|
||||
sorted.last.getName.split("\\.").last.toInt + 1
|
||||
}
|
||||
val file = sorted.find(_.lastModified() < currentTime - engineLogTimeout)
|
||||
.map { existsFile =>
|
||||
try {
|
||||
// Here we want to overwrite the exists log file
|
||||
existsFile.delete()
|
||||
existsFile.createNewFile()
|
||||
existsFile
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
warn(s"failed to delete engine log file: ${existsFile.getAbsolutePath}", e)
|
||||
null
|
||||
}
|
||||
}
|
||||
.getOrElse {
|
||||
Files.createDirectories(processLogPath)
|
||||
val newLogFile = new File(processLogPath.toFile, s"$module.log.$nextIndex")
|
||||
|
||||
@ -31,7 +31,8 @@ class ExecuteStatement(
|
||||
client: TCLIService.Iface,
|
||||
remoteSessionHandle: TSessionHandle,
|
||||
override val statement: String,
|
||||
override val shouldRunAsync: Boolean)
|
||||
override val shouldRunAsync: Boolean,
|
||||
queryTimeout: Long)
|
||||
extends KyuubiOperation(
|
||||
OperationType.EXECUTE_STATEMENT, session, client, remoteSessionHandle) {
|
||||
|
||||
@ -64,6 +65,7 @@ class ExecuteStatement(
|
||||
try {
|
||||
val req = new TExecuteStatementReq(remoteSessionHandle, statement)
|
||||
req.setRunAsync(shouldRunAsync)
|
||||
req.setQueryTimeout(queryTimeout)
|
||||
val resp = client.ExecuteStatement(req)
|
||||
verifyTStatus(resp.getStatus)
|
||||
_remoteOpHandle = resp.getOperationHandle
|
||||
|
||||
@ -22,6 +22,7 @@ import java.util.concurrent.ConcurrentHashMap
|
||||
import org.apache.hive.service.rpc.thrift.{TCLIService, TFetchResultsReq, TRow, TRowSet, TSessionHandle}
|
||||
|
||||
import org.apache.kyuubi.KyuubiSQLException
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation
|
||||
import org.apache.kyuubi.session.{Session, SessionHandle}
|
||||
import org.apache.kyuubi.util.ThriftUtils
|
||||
@ -49,6 +50,18 @@ class KyuubiOperationManager private (name: String) extends OperationManager(nam
|
||||
tSessionHandle
|
||||
}
|
||||
|
||||
private def getQueryTimeout(clientQueryTimeout: Long): Long = {
|
||||
// If clientQueryTimeout is smaller than systemQueryTimeout value,
|
||||
// we use the clientQueryTimeout value.
|
||||
val systemQueryTimeout = getConf.get(KyuubiConf.OPERATION_QUERY_TIMEOUT)
|
||||
if (clientQueryTimeout > 0 &&
|
||||
(systemQueryTimeout <= 0 || clientQueryTimeout < systemQueryTimeout)) {
|
||||
clientQueryTimeout
|
||||
} else {
|
||||
systemQueryTimeout
|
||||
}
|
||||
}
|
||||
|
||||
def setConnection(
|
||||
sessionHandle: SessionHandle,
|
||||
client: TCLIService.Iface,
|
||||
@ -69,9 +82,9 @@ class KyuubiOperationManager private (name: String) extends OperationManager(nam
|
||||
queryTimeout: Long): Operation = {
|
||||
val client = getThriftClient(session.handle)
|
||||
val remoteSessionHandle = getRemoteTSessionHandle(session.handle)
|
||||
val operation = new ExecuteStatement(session, client, remoteSessionHandle, statement, runAsync)
|
||||
val operation = new ExecuteStatement(session, client, remoteSessionHandle, statement, runAsync,
|
||||
getQueryTimeout(queryTimeout))
|
||||
addOperation(operation)
|
||||
|
||||
}
|
||||
|
||||
override def newGetTypeInfoOperation(session: Session): Operation = {
|
||||
@ -143,7 +156,6 @@ class KyuubiOperationManager private (name: String) extends OperationManager(nam
|
||||
addOperation(operation)
|
||||
}
|
||||
|
||||
|
||||
override def getOperationLogRowSet(
|
||||
opHandle: OperationHandle,
|
||||
order: FetchOrientation, maxRows: Int): TRowSet = {
|
||||
|
||||
@ -18,13 +18,15 @@
|
||||
package org.apache.kyuubi.engine.spark
|
||||
|
||||
import java.io.File
|
||||
import java.nio.file.{Files, Path, Paths}
|
||||
import java.nio.file.{Files, Path, Paths, StandardOpenOption}
|
||||
import java.time.Duration
|
||||
import java.util.concurrent.{Executors, TimeUnit}
|
||||
|
||||
import org.scalatest.time.SpanSugar._
|
||||
|
||||
import org.apache.kyuubi.{KerberizedTestHelper, KyuubiSQLException, Utils}
|
||||
import org.apache.kyuubi.config.KyuubiConf
|
||||
import org.apache.kyuubi.config.KyuubiConf.ENGINE_LOG_TIMEOUT
|
||||
import org.apache.kyuubi.service.ServiceUtils
|
||||
|
||||
class SparkProcessBuilderSuite extends KerberizedTestHelper {
|
||||
@ -179,6 +181,29 @@ class SparkProcessBuilderSuite extends KerberizedTestHelper {
|
||||
atomicTest()
|
||||
}
|
||||
}
|
||||
|
||||
test("overwrite log file should cleanup before write") {
|
||||
val fakeWorkDir = Files.createTempDirectory("fake")
|
||||
val conf = KyuubiConf()
|
||||
conf.set(ENGINE_LOG_TIMEOUT, Duration.ofDays(1).toMillis)
|
||||
val builder1 = new FakeSparkProcessBuilder(conf) {
|
||||
override val workingDir: Path = fakeWorkDir
|
||||
}
|
||||
val file1 = builder1.engineLog
|
||||
Files.write(file1.toPath, "a".getBytes(), StandardOpenOption.APPEND)
|
||||
assert(file1.length() == 1)
|
||||
Files.write(file1.toPath, "a".getBytes(), StandardOpenOption.APPEND)
|
||||
assert(file1.length() == 2)
|
||||
file1.setLastModified(System.currentTimeMillis() - Duration.ofDays(1).toMillis - 1000)
|
||||
|
||||
val builder2 = new FakeSparkProcessBuilder(conf) {
|
||||
override val workingDir: Path = fakeWorkDir
|
||||
}
|
||||
val file2 = builder2.engineLog
|
||||
assert(file1.getAbsolutePath == file2.getAbsolutePath)
|
||||
Files.write(file2.toPath, "a".getBytes(), StandardOpenOption.APPEND)
|
||||
assert(file2.length() == 1)
|
||||
}
|
||||
}
|
||||
|
||||
class FakeSparkProcessBuilder(config: KyuubiConf)
|
||||
|
||||
7
pom.xml
7
pom.xml
@ -23,7 +23,7 @@
|
||||
<groupId>org.apache.kyuubi</groupId>
|
||||
<artifactId>kyuubi</artifactId>
|
||||
<name>Kyuubi Project Parent</name>
|
||||
<version>1.1.0-SNAPSHOT</version>
|
||||
<version>1.1.0</version>
|
||||
<modules>
|
||||
<module>dev/kyuubi-codecov</module>
|
||||
<module>externals/kyuubi-download</module>
|
||||
@ -68,7 +68,7 @@
|
||||
<hadoop.version>3.2.2</hadoop.version>
|
||||
<hive.version>2.3.7</hive.version>
|
||||
<spark.version>3.0.2</spark.version>
|
||||
<spark.hadoop.binary.version>3.2</spark.hadoop.binary.version>
|
||||
<spark.hadoop.binary.version>2.7</spark.hadoop.binary.version>
|
||||
<spark.archive.name>spark-${spark.version}-bin-hadoop${spark.hadoop.binary.version}.tgz</spark.archive.name>
|
||||
<spark.archive.mirror>https://archive.apache.org/dist/spark/spark-${spark.version}</spark.archive.mirror>
|
||||
<spark.archive.download.skip>false</spark.archive.download.skip>
|
||||
@ -85,7 +85,8 @@
|
||||
|
||||
<maven.plugin.antrun.version>3.0.0</maven.plugin.antrun.version>
|
||||
<maven.plugin.compiler.version>3.8.1</maven.plugin.compiler.version>
|
||||
<maven.plugin.scala.version>4.4.0</maven.plugin.scala.version>
|
||||
<!-- DO NOT bump 4.4.0, see https://github.com/yaooqinn/kyuubi/pull/441 -->
|
||||
<maven.plugin.scala.version>4.3.0</maven.plugin.scala.version>
|
||||
<maven.plugin.surefire.version>2.22.0</maven.plugin.surefire.version>
|
||||
<maven.plugin.scalatest.version>2.0.0</maven.plugin.scalatest.version>
|
||||
<maven.plugin.scalatest.exclude.tags></maven.plugin.scalatest.exclude.tags>
|
||||
|
||||
Loading…
Reference in New Issue
Block a user