aboutsummaryrefslogtreecommitdiff
path: root/sql/hive-thriftserver/src
diff options
context:
space:
mode:
authorGuoQiang Li <witgo@qq.com>2014-08-02 13:55:28 -0700
committerPatrick Wendell <pwendell@gmail.com>2014-08-02 13:59:58 -0700
commit4c477117bb1ffef463776c86f925d35036f96b7a (patch)
tree577c6b24c177c6a5ab36929877000907c83ffee4 /sql/hive-thriftserver/src
parent91f9504e6086fac05b40545099f9818949c24bca (diff)
downloadspark-4c477117bb1ffef463776c86f925d35036f96b7a.tar.gz
spark-4c477117bb1ffef463776c86f925d35036f96b7a.tar.bz2
spark-4c477117bb1ffef463776c86f925d35036f96b7a.zip
SPARK-2804: Remove scalalogging-slf4j dependency
This also Closes #1701. Author: GuoQiang Li <witgo@qq.com> Closes #1208 from witgo/SPARK-1470 and squashes the following commits: 422646b [GuoQiang Li] Remove scalalogging-slf4j dependency
Diffstat (limited to 'sql/hive-thriftserver/src')
-rw-r--r--sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala12
-rwxr-xr-xsql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala2
-rw-r--r--sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala6
-rw-r--r--sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala6
-rw-r--r--sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala13
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala2
6 files changed, 21 insertions, 20 deletions
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
index ddbc2a79fb..08d3f983d9 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService
import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor}
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
@@ -40,7 +40,7 @@ private[hive] object HiveThriftServer2 extends Logging {
val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2")
if (!optionsProcessor.process(args)) {
- logger.warn("Error starting HiveThriftServer2 with given arguments")
+ logWarning("Error starting HiveThriftServer2 with given arguments")
System.exit(-1)
}
@@ -49,12 +49,12 @@ private[hive] object HiveThriftServer2 extends Logging {
// Set all properties specified via command line.
val hiveConf: HiveConf = ss.getConf
hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) =>
- logger.debug(s"HiveConf var: $k=$v")
+ logDebug(s"HiveConf var: $k=$v")
}
SessionState.start(ss)
- logger.info("Starting SparkContext")
+ logInfo("Starting SparkContext")
SparkSQLEnv.init()
SessionState.start(ss)
@@ -70,10 +70,10 @@ private[hive] object HiveThriftServer2 extends Logging {
val server = new HiveThriftServer2(SparkSQLEnv.hiveContext)
server.init(hiveConf)
server.start()
- logger.info("HiveThriftServer2 started")
+ logInfo("HiveThriftServer2 started")
} catch {
case e: Exception =>
- logger.error("Error starting HiveThriftServer2", e)
+ logError("Error starting HiveThriftServer2", e)
System.exit(-1)
}
}
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index cb17d7ce58..4d0c506c5a 100755
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.shims.ShimLoader
import org.apache.thrift.transport.TSocket
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
private[hive] object SparkSQLCLIDriver {
private var prompt = "spark-sql"
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index a56b19a4bc..d362d599d0 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema}
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext)
@@ -40,7 +40,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo
private def getResultSetSchema(query: context.QueryExecution): Schema = {
val analyzed = query.analyzed
- logger.debug(s"Result Schema: ${analyzed.output}")
+ logDebug(s"Result Schema: ${analyzed.output}")
if (analyzed.output.size == 0) {
new Schema(new FieldSchema("Response code", "string", "") :: Nil, null)
} else {
@@ -61,7 +61,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo
new CommandProcessorResponse(0)
} catch {
case cause: Throwable =>
- logger.error(s"Failed in [$command]", cause)
+ logError(s"Failed in [$command]", cause)
new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null)
}
}
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala
index 451c3bd7b9..582264eb59 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala
@@ -20,13 +20,13 @@ package org.apache.spark.sql.hive.thriftserver
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.spark.scheduler.{SplitInfo, StatsReportListener}
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.{SparkConf, SparkContext}
/** A singleton object for the master program. The slaves should not access this. */
private[hive] object SparkSQLEnv extends Logging {
- logger.debug("Initializing SparkSQLEnv")
+ logDebug("Initializing SparkSQLEnv")
var hiveContext: HiveContext = _
var sparkContext: SparkContext = _
@@ -47,7 +47,7 @@ private[hive] object SparkSQLEnv extends Logging {
/** Cleans up and shuts down the Spark SQL environments. */
def stop() {
- logger.debug("Shutting down Spark SQL Environment")
+ logDebug("Shutting down Spark SQL Environment")
// Stop the SparkContext
if (SparkSQLEnv.sparkContext != null) {
sparkContext.stop()
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
index a4e1f3e762..d4dadfd21d 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
@@ -30,10 +30,11 @@ import org.apache.hive.service.cli._
import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager}
import org.apache.hive.service.cli.session.HiveSession
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.hive.thriftserver.ReflectionUtils
import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
-import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow}
+import org.apache.spark.sql.{SchemaRDD, Row => SparkRow}
/**
* Executes queries using Spark SQL, and maintains a list of handles to active queries.
@@ -55,7 +56,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage
def close(): Unit = {
// RDDs will be cleaned automatically upon garbage collection.
- logger.debug("CLOSING")
+ logDebug("CLOSING")
}
def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = {
@@ -112,7 +113,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage
}
def getResultSetSchema: TableSchema = {
- logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}")
+ logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}")
if (result.queryExecution.analyzed.output.size == 0) {
new TableSchema(new FieldSchema("Result", "string", "") :: Nil)
} else {
@@ -124,11 +125,11 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage
}
def run(): Unit = {
- logger.info(s"Running query '$statement'")
+ logInfo(s"Running query '$statement'")
setState(OperationState.RUNNING)
try {
result = hiveContext.hql(statement)
- logger.debug(result.queryExecution.toString())
+ logDebug(result.queryExecution.toString())
val groupId = round(random * 1000000).toString
hiveContext.sparkContext.setJobGroup(groupId, statement)
iter = result.queryExecution.toRdd.toLocalIterator
@@ -138,7 +139,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage
// Actually do need to catch Throwable as some failures don't inherit from Exception and
// HiveServer will silently swallow them.
case e: Throwable =>
- logger.error("Error executing query:",e)
+ logError("Error executing query:",e)
throw new HiveSQLException(e.toString)
}
setState(OperationState.FINISHED)
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
index fe3403b329..b7b7c9957a 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala
@@ -27,7 +27,7 @@ import java.sql.{Connection, DriverManager, Statement}
import org.scalatest.{BeforeAndAfterAll, FunSuite}
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.util.getTempFilePath
/**