From dab37966b0cfd290919ca5c005f59dde00615c0e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 1 Aug 2014 23:55:30 -0700 Subject: Revert "[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api" This reverts commit adc8303294e26efb4ed15e5f5ba1062f7988625d. --- .../spark/sql/hive/thriftserver/HiveThriftServer2.scala | 12 ++++++------ .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala | 6 +++--- .../apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala | 6 +++--- .../hive/thriftserver/server/SparkSQLOperationManager.scala | 13 ++++++------- .../sql/hive/thriftserver/HiveThriftServer2Suite.scala | 2 +- 6 files changed, 20 insertions(+), 21 deletions(-) (limited to 'sql/hive-thriftserver') 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 5959ba3d23..ddbc2a79fb 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.Logging +import org.apache.spark.sql.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)) { - log.warn("Error starting HiveThriftServer2 with given arguments") + logger.warn("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) => - log.debug(s"HiveConf var: $k=$v") + logger.debug(s"HiveConf var: $k=$v") } SessionState.start(ss) - log.info("Starting SparkContext") + logger.info("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() - log.info("HiveThriftServer2 started") + logger.info("HiveThriftServer2 started") } catch { case e: Exception => - log.error("Error starting HiveThriftServer2", e) + logger.error("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 4d0c506c5a..cb17d7ce58 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.Logging +import org.apache.spark.sql.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 276723990b..a56b19a4bc 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.Logging +import org.apache.spark.sql.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 - log.debug(s"Result Schema: ${analyzed.output}") + logger.debug(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 => - log.error(s"Failed in [$command]", cause) + logger.error(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 dfc93b19d0..451c3bd7b9 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.Logging +import org.apache.spark.sql.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 { - log.debug("Initializing SparkSQLEnv") + logger.debug("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() { - log.debug("Shutting down Spark SQL Environment") + logger.debug("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 2c6e24e80d..a4e1f3e762 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,11 +30,10 @@ 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.{SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -56,7 +55,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. - log.debug("CLOSING") + logger.debug("CLOSING") } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { @@ -113,7 +112,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - log.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { @@ -125,11 +124,11 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def run(): Unit = { - log.info(s"Running query '$statement'") + logger.info(s"Running query '$statement'") setState(OperationState.RUNNING) try { result = hiveContext.hql(statement) - log.debug(result.queryExecution.toString()) + logger.debug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) iter = result.queryExecution.toRdd.toLocalIterator @@ -139,7 +138,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 => - log.error("Error executing query:",e) + logger.error("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 b7b7c9957a..fe3403b329 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.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** -- cgit v1.2.3