Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -480,7 +480,8 @@ object SparkParallelTestGrouping {
"org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite",
"org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite",
"org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite",
"org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite",
"org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInHttpSuite",
"org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextInBinarySuite",
Copy link
Member

@dongjoon-hyun dongjoon-hyun Jun 10, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks like a SBT-only workaround, doesn't it?

"org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does that mean this approach to speed up the test runner never works for maven? cc @gengliangwang @wangyum

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it's not related to maven I think.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree with @gengliangwang.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any way to run these test JVM-individually with maven? It seems not to be able to start 2 thrift servers with different kinds of transport modes on the shared spark session in one JVM

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we just run these 2 test suites one by one?

Copy link
Member Author

@yaooqinn yaooqinn Jun 11, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The root cause I found so far is: in afterAll(), the spark session was stopped and detached from the thread-local variable, but the hive's SessionState was not, SO it gets reused next time which causes the newly defined configs in the new test file will not take effect.

If what I‘ve found is the only issue that stops these tests from running together in a single JVM(verified locally and went well), I guess we can remove these 2 lines eventually.

Sended a new PR, #28797

)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@ trait SharedThriftServer extends SharedSparkSession {
private var hiveServer2: HiveThriftServer2 = _
private var serverPort: Int = 0

def mode: ServerMode.Value

override def beforeAll(): Unit = {
super.beforeAll()
// Retries up to 3 times with different port numbers if the server fails to start
Expand All @@ -53,11 +55,17 @@ trait SharedThriftServer extends SharedSparkSession {
}
}

protected def jdbcUri: String = if (mode == ServerMode.http) {
s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice"
} else {
s"jdbc:hive2://localhost:$serverPort"
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit format:

private lazy val jdbcUri = if (mode == ServerMode.http) {
    s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice"
  } else {
    s"jdbc:hive2://localhost:$serverPort"
  }

?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the existing format is correct.


protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = {
val user = System.getProperty("user.name")
require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2")
val connections =
fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") }
fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") }
val statements = connections.map(_.createStatement())

try {
Expand All @@ -71,21 +79,33 @@ trait SharedThriftServer extends SharedSparkSession {
private def startThriftServer(attempt: Int): Unit = {
logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt")
val sqlContext = spark.newSession().sqlContext
// Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use.
// Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could
// randomly pick any free port to use.
// It's much more robust than set a random port generated by ourselves ahead
sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0")
hiveServer2 = HiveThriftServer2.startWithContext(sqlContext)
hiveServer2.getServices.asScala.foreach {
case t: ThriftCLIService if t.getPortNumber != 0 =>
serverPort = t.getPortNumber
logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt")
case _ =>
}
sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0")
sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString)

try {
hiveServer2 = HiveThriftServer2.startWithContext(sqlContext)
hiveServer2.getServices.asScala.foreach {
case t: ThriftCLIService =>
serverPort = t.getPortNumber
logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so we may not output this log?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Before this fix, yes. The port binding is in another background thread.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how does this patch fix it? It seems you just added a try-catch?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With https://github.com/apache/spark/pull/28751/files#diff-7610697b4f8f1bc4842c77e50807914cR178 and its implementations, the port binding is done in the same thread where we call getPortNumber later.

Copy link
Member Author

@yaooqinn yaooqinn Jun 8, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#28651 (comment) . there was a discussion with @juliuszsompolski before

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah I see!

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Take ThriftBinaryCLIService for an example
Before:
we do TThreadPoolServer initialization and serve in the same run function of the background thread. Then if we call getPortNumber right after startWithContext, concurrency issue will occur. The portNum may not reset yet when we call.

After:
we do TThreadPoolServer initialization in the current thread and do serve in the run function of the background thread.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, I see. Nice catch.

case _ =>
}

// Wait for thrift server to be ready to serve the query, via executing simple query
// till the query succeeds. See SPARK-30345 for more details.
eventually(timeout(30.seconds), interval(1.seconds)) {
withJdbcStatement { _.execute("SELECT 1") }
// Wait for thrift server to be ready to serve the query, via executing simple query
// till the query succeeds. See SPARK-30345 for more details.
eventually(timeout(30.seconds), interval(1.seconds)) {
withJdbcStatement { _.execute("SELECT 1") }
}
} catch {
case e: Exception =>
logError("Error start hive server with Context ", e)
if (hiveServer2 != null) {
hiveServer2.stop()
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,9 @@ import org.apache.spark.sql.types._
*/
class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer {


override def mode: ServerMode.Value = ServerMode.binary

override protected def testFile(fileName: String): String = {
val url = Thread.currentThread().getContextClassLoader.getResource(fileName)
// Copy to avoid URISyntaxException during accessing the resources in `sql/core`
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.hive.thriftserver

class ThriftServerWithSparkContextSuite extends SharedThriftServer {
trait ThriftServerWithSparkContextSuite extends SharedThriftServer {

test("SPARK-29911: Uncache cached tables when session closed") {
val cacheManager = spark.sharedState.cacheManager
Expand All @@ -42,3 +42,12 @@ class ThriftServerWithSparkContextSuite extends SharedThriftServer {
}
}
}


class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

override def mode: ServerMode.Value = ServerMode.binary
}

class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite {
override def mode: ServerMode.Value = ServerMode.http
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hive.service.ServiceException;
import org.apache.hive.service.auth.HiveAuthFactory;
import org.apache.hive.service.cli.CLIService;
import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup;
Expand All @@ -45,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) {
}

@Override
public void run() {
protected void initializeServer() {
try {
// Server thread pool
String threadPoolName = "HiveServer2-Handler-Pool";
Expand Down Expand Up @@ -100,6 +101,14 @@ public void run() {
String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port "
+ serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads";
LOG.info(msg);
} catch (Exception t) {
throw new ServiceException("Error initializing " + getName(), t);
}
}

@Override
public void run() {
try {
server.serve();
} catch (Throwable t) {
LOG.fatal(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,7 @@ public synchronized void init(HiveConf hiveConf) {
public synchronized void start() {
super.start();
if (!isStarted && !isEmbedded) {
initializeServer();
new Thread(this).start();
isStarted = true;
}
Expand Down Expand Up @@ -633,6 +634,8 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException {
return resp;
}

protected abstract void initializeServer();

@Override
public abstract void run();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Shell;
import org.apache.hive.service.ServiceException;
import org.apache.hive.service.auth.HiveAuthFactory;
import org.apache.hive.service.cli.CLIService;
import org.apache.hive.service.cli.thrift.TCLIService.Iface;
Expand All @@ -53,13 +54,8 @@ public ThriftHttpCLIService(CLIService cliService) {
super(cliService, ThriftHttpCLIService.class.getSimpleName());
}

/**
* Configure Jetty to serve http requests. Example of a client connection URL:
* http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ,
* e.g. http://gateway:port/hive2/servlets/thrifths2/
*/
@Override
public void run() {
protected void initializeServer() {
try {
// Server thread pool
// Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests
Expand Down Expand Up @@ -150,6 +146,19 @@ public void run() {
+ " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..."
+ maxWorkerThreads + " worker threads";
LOG.info(msg);
} catch (Exception t) {
throw new ServiceException("Error initializing " + getName(), t);
}
}

/**
* Configure Jetty to serve http requests. Example of a client connection URL:
* http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ,
* e.g. http://gateway:port/hive2/servlets/thrifths2/
*/
@Override
public void run() {
try {
httpServer.join();
} catch (Throwable t) {
LOG.fatal(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hive.service.ServiceException;
import org.apache.hive.service.auth.HiveAuthFactory;
import org.apache.hive.service.cli.CLIService;
import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup;
Expand All @@ -46,7 +47,7 @@ public ThriftBinaryCLIService(CLIService cliService) {
}

@Override
public void run() {
protected void initializeServer() {
try {
// Server thread pool
String threadPoolName = "HiveServer2-Handler-Pool";
Expand Down Expand Up @@ -101,6 +102,14 @@ public void run() {
String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port "
+ portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads";
LOG.info(msg);
} catch (Exception t) {
throw new ServiceException("Error initializing " + getName(), t);
}
}

@Override
public void run() {
try {
server.serve();
} catch (Throwable t) {
LOG.error(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,7 @@ public synchronized void init(HiveConf hiveConf) {
public synchronized void start() {
super.start();
if (!isStarted && !isEmbedded) {
initializeServer();
new Thread(this).start();
isStarted = true;
}
Expand Down Expand Up @@ -670,6 +671,8 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req)
return resp;
}

protected abstract void initializeServer();

@Override
public abstract void run();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Shell;
import org.apache.hive.service.ServiceException;
import org.apache.hive.service.auth.HiveAuthFactory;
import org.apache.hive.service.cli.CLIService;
import org.apache.hive.service.rpc.thrift.TCLIService;
Expand All @@ -54,13 +55,8 @@ public ThriftHttpCLIService(CLIService cliService) {
super(cliService, ThriftHttpCLIService.class.getSimpleName());
}

/**
* Configure Jetty to serve http requests. Example of a client connection URL:
* http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ,
* e.g. http://gateway:port/hive2/servlets/thrifths2/
*/
@Override
public void run() {
protected void initializeServer() {
try {
// Server thread pool
// Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests
Expand Down Expand Up @@ -151,6 +147,19 @@ public void run() {
+ " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..."
+ maxWorkerThreads + " worker threads";
LOG.info(msg);
} catch (Exception t) {
throw new ServiceException("Error initializing " + getName(), t);
}
}

/**
* Configure Jetty to serve http requests. Example of a client connection URL:
* http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ,
* e.g. http://gateway:port/hive2/servlets/thrifths2/
*/
@Override
public void run() {
try {
httpServer.join();
} catch (Throwable t) {
LOG.error(
Expand Down