diff --git a/LICENSE-binary b/LICENSE-binary index d363661b1cc7e..2a5434e14a3f5 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -521,7 +521,6 @@ Common Development and Distribution License (CDDL) 1.1 ------------------------------------------------------ javax.el:javax.el-api https://javaee.github.io/uel-ri/ -javax.servlet:javax.servlet-api https://javaee.github.io/servlet-spec/ javax.servlet.jsp:jsp-api javax.transaction:jta http://www.oracle.com/technetwork/java/index.html javax.xml.bind:jaxb-api https://github.com/javaee/jaxb-v2 @@ -553,6 +552,7 @@ Eclipse Public License (EPL) 2.0 -------------------------------- jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca +jakarta.servlet:jakarta.servlet-api https://projects.eclipse.org/projects/ee4j.servlet jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api org.glassfish.hk2.external:jakarta.inject diff --git a/core/pom.xml b/core/pom.xml index 84ca852d1f30a..1f24c5273ad0b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -161,9 +161,9 @@ compile - javax.servlet - javax.servlet-api - ${javaxservlet.version} + jakarta.servlet + jakarta.servlet-api + ${jakartaservlet.version} org.apache.commons diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index c2caef3ae58d9..ceea496d3f1dc 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -119,6 +119,7 @@ jackson-xc/1.9.13//jackson-xc-1.9.13.jar jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.servlet-api/4.0.3//jakarta.servlet-api-4.0.3.jar jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar @@ -126,7 +127,6 @@ janino/3.0.16//janino-3.0.16.jar javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar -javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar javolution/5.5.1//javolution-5.5.1.jar jaxb-api/2.2.2//jaxb-api-2.2.2.jar jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 87e7a3c2ae1a7..d1b811bd73607 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -118,6 +118,7 @@ jackson-module-scala_2.12/2.11.4//jackson-module-scala_2.12-2.11.4.jar jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.servlet-api/4.0.3//jakarta.servlet-api-4.0.3.jar jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar diff --git a/pom.xml b/pom.xml index 09d48a6592ab9..78d1fe7d54350 100644 --- a/pom.xml +++ b/pom.xml @@ -138,7 +138,7 @@ 1.10.1 1.6.6 9.4.28.v20200408 - 3.1.0 + 4.0.3 0.9.5 2.4.0 2.0.8 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 23fb73d228e01..a28c2b55b3789 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -395,6 +395,8 @@ object SparkBuild extends PomBuild { enable(KubernetesIntegrationTests.settings)(kubernetesIntegrationTests) + enable(YARN.settings)(yarn) + /** * Adds the ability to run the spark shell directly from SBT without building an assembly * jar. @@ -654,7 +656,21 @@ object DependencyOverrides { */ object ExcludedDependencies { lazy val settings = Seq( - libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") } + libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") }, + // SPARK-33705: Due to sbt compiler issues, it brings exclusions defined in maven pom back to + // the classpath directly and assemble test scope artifacts to assembly/target/scala-xx/jars, + // which is also will be added to the classpath of some unit tests that will build a subprocess + // to run `spark-submit`, e.g. HiveThriftServer2Test. + // + // These artifacts are for the jersey-1 API but Spark use jersey-2 ones, so it cause test + // flakiness w/ jar conflicts issues. + // + // Also jersey-1 is only used by yarn module(see resource-managers/yarn/pom.xml) for testing + // purpose only. Here we exclude them from the whole project scope and add them w/ yarn only. + excludeDependencies ++= Seq( + ExclusionRule(organization = "com.sun.jersey"), + ExclusionRule("javax.servlet", "javax.servlet-api"), + ExclusionRule("javax.ws.rs", "jsr311-api")) ) } @@ -758,6 +774,15 @@ object Hive { ) } +object YARN { + lazy val settings = Seq( + excludeDependencies --= Seq( + ExclusionRule(organization = "com.sun.jersey"), + ExclusionRule("javax.servlet", "javax.servlet-api"), + ExclusionRule("javax.ws.rs", "jsr311-api")) + ) +} + object Assembly { import sbtassembly.AssemblyUtils._ import sbtassembly.AssemblyPlugin.autoImport._ diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 1d3856742f520..c0ce1c8e151ed 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -88,13 +88,6 @@ hadoop-client - - jakarta.servlet - jakarta.servlet-api - 4.0.3 - test - - com.google.guava diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 5bf7892478082..bd0db743b8d4c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -29,7 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future, Promise} import scala.concurrent.duration._ import scala.io.Source -import scala.util.{Random, Try} +import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.hive.conf.HiveConf.ConfVars @@ -41,6 +41,7 @@ import org.apache.hive.service.rpc.thrift.TCLIService.Client import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging @@ -60,7 +61,7 @@ object TestData { val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") } -class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { +class HiveThriftBinaryServerSuite extends HiveThriftServer2Test { override def mode: ServerMode.Value = ServerMode.binary private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { @@ -935,7 +936,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } -class SingleSessionSuite extends HiveThriftJdbcTest { +class SingleSessionSuite extends HiveThriftServer2TestBase { override def mode: ServerMode.Value = ServerMode.binary override protected def extraConf: Seq[String] = @@ -1046,7 +1047,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } } -class HiveThriftCleanUpScratchDirSuite extends HiveThriftJdbcTest{ +class HiveThriftCleanUpScratchDirSuite extends HiveThriftServer2TestBase { var tempScratchDir: File = _ override protected def beforeAll(): Unit = { @@ -1079,7 +1080,7 @@ class HiveThriftCleanUpScratchDirSuite extends HiveThriftJdbcTest{ } } -class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { +class HiveThriftHttpServerSuite extends HiveThriftServer2Test { override def mode: ServerMode.Value = ServerMode.http test("JDBC query execution") { @@ -1122,63 +1123,7 @@ object ServerMode extends Enumeration { val binary, http = Value } -abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { - Utils.classForName(classOf[HiveDriver].getCanonicalName) - - private def jdbcUri = if (mode == ServerMode.http) { - s"""jdbc:hive2://localhost:$serverPort/ - |default? - |hive.server2.transport.mode=http; - |hive.server2.thrift.http.path=cliservice; - |${hiveConfList}#${hiveVarList} - """.stripMargin.split("\n").mkString.trim - } else { - s"jdbc:hive2://localhost:$serverPort/?${hiveConfList}#${hiveVarList}" - } - - def withMultipleConnectionJdbcStatement(tableNames: String*)(fs: (Statement => Unit)*): Unit = { - val user = System.getProperty("user.name") - val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } - val statements = connections.map(_.createStatement()) - - try { - statements.zip(fs).foreach { case (s, f) => f(s) } - } finally { - tableNames.foreach { name => - // TODO: Need a better way to drop the view. - if (name.toUpperCase(Locale.ROOT).startsWith("VIEW")) { - statements(0).execute(s"DROP VIEW IF EXISTS $name") - } else { - statements(0).execute(s"DROP TABLE IF EXISTS $name") - } - } - statements.foreach(_.close()) - connections.foreach(_.close()) - } - } - - def withDatabase(dbNames: String*)(fs: (Statement => Unit)*): Unit = { - val user = System.getProperty("user.name") - val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } - val statements = connections.map(_.createStatement()) - - try { - statements.zip(fs).foreach { case (s, f) => f(s) } - } finally { - dbNames.foreach { name => - statements(0).execute(s"DROP DATABASE IF EXISTS $name") - } - statements.foreach(_.close()) - connections.foreach(_.close()) - } - } - - def withJdbcStatement(tableNames: String*)(f: Statement => Unit): Unit = { - withMultipleConnectionJdbcStatement(tableNames: _*)(f) - } -} - -abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAll with Logging { +abstract class HiveThriftServer2TestBase extends SparkFunSuite with BeforeAndAfterAll with Logging { def mode: ServerMode.Value private val CLASS_NAME = HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$") @@ -1207,7 +1152,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl protected def extraConf: Seq[String] = Nil - protected def serverStartCommand(port: Int) = { + protected def serverStartCommand(): Seq[String] = { val portConf = if (mode == ServerMode.binary) { ConfVars.HIVE_SERVER2_THRIFT_PORT } else { @@ -1220,7 +1165,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl val tempLog4jConf = Utils.createTempDir().getCanonicalPath Files.write( - """log4j.rootCategory=DEBUG, console + """log4j.rootCategory=INFO, console |log4j.appender.console=org.apache.log4j.ConsoleAppender |log4j.appender.console.target=System.err |log4j.appender.console.layout=org.apache.log4j.PatternLayout @@ -1240,7 +1185,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=$mode | --hiveconf ${ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION}=$operationLogPath | --hiveconf ${ConfVars.LOCALSCRATCHDIR}=$lScratchDir - | --hiveconf $portConf=$port + | --hiveconf $portConf=0 | --driver-class-path $driverClassPath | --driver-java-options -Dlog4j.debug | --conf spark.ui.enabled=false @@ -1262,7 +1207,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl val SERVER_STARTUP_TIMEOUT = 3.minutes - private def startThriftServer(port: Int, attempt: Int) = { + private def startThriftServer(attempt: Int) = { warehousePath = Utils.createTempDir() warehousePath.delete() metastorePath = Utils.createTempDir() @@ -1274,18 +1219,16 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl logPath = null logTailingProcess = null - val command = serverStartCommand(port) + val command = serverStartCommand() diagnosisBuffer ++= s""" |### Attempt $attempt ### |HiveThriftServer2 command line: $command - |Listening port: $port + |Listening port: 0 |System user: $user """.stripMargin.split("\n") - logInfo(s"Trying to start HiveThriftServer2: port=$port, mode=$mode, attempt=$attempt") - logPath = { val lines = Utils.executeAndGetOutput( command = command, @@ -1312,7 +1255,11 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl // Ensures that the following "tail" command won't fail. logPath.createNewFile() - val successLines = Seq(THRIFT_BINARY_SERVICE_LIVE, THRIFT_HTTP_SERVICE_LIVE) + val successLine = if (mode == ServerMode.http) { + THRIFT_HTTP_SERVICE_LIVE + } else { + THRIFT_BINARY_SERVICE_LIVE + } logTailingProcess = { val command = s"/usr/bin/env tail -n +0 -f ${logPath.getCanonicalPath}".split(" ") @@ -1321,14 +1268,15 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl val captureOutput = (line: String) => diagnosisBuffer.synchronized { diagnosisBuffer += line - successLines.foreach { r => - if (line.contains(r)) { - serverStarted.trySuccess(()) - } + if (line.contains(successLine)) { + listeningPort = line.split(" on port ")(1).split(' ').head.toInt + logInfo(s"Started HiveThriftServer2: port=$listeningPort, mode=$mode, attempt=$attempt") + serverStarted.trySuccess(()) + () } } - val process = builder.start() + val process = builder.start() new ProcessOutputCapturer(process.getInputStream, captureOutput).start() new ProcessOutputCapturer(process.getErrorStream, captureOutput).start() @@ -1379,16 +1327,18 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl override protected def beforeAll(): Unit = { super.beforeAll() - // Chooses a random port between 10000 and 19999 - listeningPort = 10000 + Random.nextInt(10000) diagnosisBuffer.clear() // Retries up to 3 times with different port numbers if the server fails to start - (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) => + (1 to 3).foldLeft(Try(startThriftServer(0))) { case (started, attempt) => started.orElse { - listeningPort += 1 stopThriftServer() - Try(startThriftServer(listeningPort, attempt)) + Try { + startThriftServer(attempt) + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement() { _.execute("SELECT 1") } + } + } } }.recover { case cause: Throwable => @@ -1407,4 +1357,91 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl super.afterAll() } } + + Utils.classForName(classOf[HiveDriver].getCanonicalName) + + protected def jdbcUri(database: String = "default"): String = if (mode == ServerMode.http) { + s"""jdbc:hive2://localhost:$serverPort/ + |$database? + |hive.server2.transport.mode=http; + |hive.server2.thrift.http.path=cliservice; + |${hiveConfList}#${hiveVarList} + """.stripMargin.split("\n").mkString.trim + } else { + s"jdbc:hive2://localhost:$serverPort/$database?${hiveConfList}#${hiveVarList}" + } + + private def tryCaptureSysLog(f: => Unit): Unit = { + try f catch { + case e: Exception => + // Dump the HiveThriftServer2 log if error occurs, e.g. getConnection failure. + dumpLogs() + throw e + } + } + + def withMultipleConnectionJdbcStatement( + tableNames: String*)(fs: (Statement => Unit)*): Unit = tryCaptureSysLog { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri(), user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + tableNames.foreach { name => + // TODO: Need a better way to drop the view. + if (name.toUpperCase(Locale.ROOT).startsWith("VIEW")) { + statements(0).execute(s"DROP VIEW IF EXISTS $name") + } else { + statements(0).execute(s"DROP TABLE IF EXISTS $name") + } + } + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } + + def withDatabase(dbNames: String*)(fs: (Statement => Unit)*): Unit = tryCaptureSysLog { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri(), user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + dbNames.foreach { name => + statements(0).execute(s"DROP DATABASE IF EXISTS $name") + } + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } + + def withJdbcStatement(tableNames: String*)(f: Statement => Unit): Unit = { + withMultipleConnectionJdbcStatement(tableNames: _*)(f) + } +} + +/** + * Common tests for both binary and http mode thrift server + * TODO: SPARK-31914: Move common tests from subclasses to this trait + */ +abstract class HiveThriftServer2Test extends HiveThriftServer2TestBase { + test("SPARK-17819: Support default database in connection URIs") { + withDatabase("spark17819") { statement => + statement.execute(s"CREATE DATABASE IF NOT EXISTS spark17819") + val jdbcStr = jdbcUri("spark17819") + val connection = DriverManager.getConnection(jdbcStr, user, "") + val statementN = connection.createStatement() + try { + val resultSet = statementN.executeQuery("select current_database()") + resultSet.next() + assert(resultSet.getString(1) === "spark17819") + } finally { + statementN.close() + connection.close() + } + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala deleted file mode 100644 index fb8a7e273ae44..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala +++ /dev/null @@ -1,70 +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.spark.sql.hive.thriftserver - -import java.sql.DriverManager - -import org.apache.hive.jdbc.HiveDriver - -import org.apache.spark.util.Utils - -class JdbcConnectionUriSuite extends HiveThriftServer2Test { - Utils.classForName(classOf[HiveDriver].getCanonicalName) - - override def mode: ServerMode.Value = ServerMode.binary - - val JDBC_TEST_DATABASE = "jdbc_test_database" - val USER = System.getProperty("user.name") - val PASSWORD = "" - - override protected def beforeAll(): Unit = { - super.beforeAll() - - val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" - val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) - val statement = connection.createStatement() - statement.execute(s"CREATE DATABASE $JDBC_TEST_DATABASE") - connection.close() - } - - override protected def afterAll(): Unit = { - try { - val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" - val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) - val statement = connection.createStatement() - statement.execute(s"DROP DATABASE $JDBC_TEST_DATABASE") - connection.close() - } finally { - super.afterAll() - } - } - - test("SPARK-17819 Support default database in connection URIs") { - val jdbcUri = s"jdbc:hive2://localhost:$serverPort/$JDBC_TEST_DATABASE" - val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) - val statement = connection.createStatement() - try { - val resultSet = statement.executeQuery("select current_database()") - resultSet.next() - assert(resultSet.getString(1) === JDBC_TEST_DATABASE) - } finally { - statement.close() - connection.close() - } - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index b413b46adcaa1..bb7448293f559 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.types._ import org.apache.spark.util.VersionUtils -class SparkMetadataOperationSuite extends HiveThriftJdbcTest { +class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { override def mode: ServerMode.Value = ServerMode.binary diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index 52cf429441d16..fd4d7231e8989 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -31,7 +31,7 @@ import org.apache.thrift.transport.TSocket import org.apache.spark.sql.catalyst.util.NumberConverter import org.apache.spark.unsafe.types.UTF8String -class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { +class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase { override def mode: ServerMode.Value = ServerMode.binary diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index d0b829c240327..2d0edb8eb8d48 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -32,7 +32,7 @@ import org.scalatestplus.selenium.WebBrowser import org.apache.spark.ui.SparkUICssErrorHandler class UISeleniumSuite - extends HiveThriftJdbcTest + extends HiveThriftServer2TestBase with WebBrowser with Matchers with BeforeAndAfterAll { implicit var webDriver: WebDriver = _ @@ -57,7 +57,7 @@ class UISeleniumSuite } } - override protected def serverStartCommand(port: Int) = { + override protected def serverStartCommand(): Seq[String] = { val portConf = if (mode == ServerMode.binary) { ConfVars.HIVE_SERVER2_THRIFT_PORT } else { @@ -71,7 +71,7 @@ class UISeleniumSuite | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=$mode - | --hiveconf $portConf=$port + | --hiveconf $portConf=0 | --driver-class-path ${sys.props("java.class.path")} | --conf spark.ui.enabled=true | --conf spark.ui.port=$uiPort