Skip to content

Commit

Permalink
println the process output
Browse files Browse the repository at this point in the history
  • Loading branch information
scwf committed Oct 6, 2014
1 parent 0081a50 commit 48979f6
Showing 1 changed file with 14 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hive.jdbc.HiveDriver
import org.scalatest.{BeforeAndAfterAll, FunSuite}

import org.apache.spark.Logging
import org.apache.spark.{SparkException, Logging}
import org.apache.spark.sql.catalyst.util._

/**
Expand All @@ -58,7 +58,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi
val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/"
val user = System.getProperty("user.name")

override def beforeAll(): Unit = {
override def beforeAll() = {
val timeout: FiniteDuration = 30.seconds
val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator)
val command =
Expand Down Expand Up @@ -87,8 +87,10 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi
// logFile may have not finished, try every second
while (!logFile.exists() || (!fileToString(logFile).contains(
"ThriftBinaryCLIService listening on") && tryNum < maxTries)) {
tryNum = tryNum + 1
Thread.sleep(1000)
}
println(fileToString(logFile))
if (fileToString(logFile).contains("ThriftBinaryCLIService listening on")) {
serverStarted.success(())
} else {
Expand All @@ -101,7 +103,8 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi

Future {
val exitValue = process.exitValue()
logInfo(s"Start Spark SQL Thrift server process exit value: $exitValue")
serverStarted.tryFailure(
new SparkException(s"Spark SQL Thrift server process exit value: $exitValue"))
}

try {
Expand All @@ -127,6 +130,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi
|End HiveThriftServer2Suite failure output
|=========================================
""".stripMargin, cause)
throw cause
} finally {
process.destroy()
}
Expand All @@ -138,26 +142,24 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi
stopThriftserver
}

def stopThriftserver: Unit = {
def stopThriftserver = {
val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator)
val builder = new ProcessBuilder(stopScript)
val process = builder.start()
new Thread("read stderr") {
override def run() {
for (line <- Source.fromInputStream(process.getErrorStream).getLines()) {
System.err.println(line)
println(line)
}
}
}.start()
val output = new StringBuffer
val stdoutThread = new Thread("read stdout") {
new Thread("read stdout") {
override def run() {
for (line <- Source.fromInputStream(process.getInputStream).getLines()) {
output.append(line)
println(line)
}
}
}
stdoutThread.start()
}.start()
val exitValue = process.waitFor()
logInfo(s"Stop Spark SQL Thrift server process exit value: $exitValue")
}
Expand All @@ -180,6 +182,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi
assertResult(5, "Row count mismatch") {
val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test")
resultSet.next()
// println(s"#######${resultSet.getInt(1)}")
resultSet.getInt(1)
}
} finally {
Expand Down Expand Up @@ -208,6 +211,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with Loggi

(0 until 5).foreach { _ =>
resultSet.next()
// println(s"#######${resultSet.getInt(1)}")
assert(resultSet.getInt(1) === 0)
assert(resultSet.wasNull())
}
Expand Down

0 comments on commit 48979f6

Please sign in to comment.