diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 807ef3e9c9d60376d362a24220d66d92c2d6c814..d4f2624061e35f4ccb3750fc84945b3ae6af5b34 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -39,13 +39,17 @@ trait Logging {
   // be serialized and used on another machine
   @transient private var log_ : Logger = null
 
+  // Method to get the logger name for this object
+  protected def logName = {
+    // Ignore trailing $'s in the class names for Scala objects
+    this.getClass.getName.stripSuffix("$")
+  }
+
   // Method to get or create the logger for this object
   protected def log: Logger = {
     if (log_ == null) {
       initializeIfNecessary()
-      var className = this.getClass.getName
-      // Ignore trailing $'s in the class names for Scala objects
-      log_ = LoggerFactory.getLogger(className.stripSuffix("$"))
+      log_ = LoggerFactory.getLogger(logName)
     }
     log_
   }
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 54fa96baa1e1833c01dbe612c8cd8cce492fb531..58d44e7923bee027212cf33f4d309b0870cf1d6c 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -54,11 +54,6 @@
       <artifactId>spark-core_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
-    <dependency>
-      <groupId>com.typesafe</groupId>
-      <artifactId>scalalogging-slf4j_${scala.binary.version}</artifactId>
-      <version>1.0.1</version>
-    </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 74c0104e5b17f83b84aa8edc9e3e938f9fe32841..2ba68cab115fb4be1ac179614d5c7519a83b0426 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
   object ResolveReferences extends Rule[LogicalPlan] {
     def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
       case q: LogicalPlan if q.childrenResolved =>
-        logger.trace(s"Attempting to resolve ${q.simpleString}")
+        logTrace(s"Attempting to resolve ${q.simpleString}")
         q transformExpressions {
           case u @ UnresolvedAttribute(name) =>
             // Leave unchanged if resolution fails.  Hopefully will be resolved next round.
             val result = q.resolve(name).getOrElse(u)
-            logger.debug(s"Resolving $u to $result")
+            logDebug(s"Resolving $u to $result")
             result
         }
     }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 47c7ad076ad076aeeb127fca812a21968357afce..e94f2a3bea63ea850ea37fedfb48198b9b1e0171 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -75,7 +75,7 @@ trait HiveTypeCoercion {
             // Leave the same if the dataTypes match.
             case Some(newType) if a.dataType == newType.dataType => a
             case Some(newType) =>
-              logger.debug(s"Promoting $a to $newType in ${q.simpleString}}")
+              logDebug(s"Promoting $a to $newType in ${q.simpleString}}")
               newType
           }
       }
@@ -154,7 +154,7 @@ trait HiveTypeCoercion {
             (Alias(Cast(l, StringType), l.name)(), r)
 
           case (l, r) if l.dataType != r.dataType =>
-            logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
+            logDebug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
             findTightestCommonType(l.dataType, r.dataType).map { widestType =>
               val newLeft =
                 if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)()
@@ -170,7 +170,7 @@ trait HiveTypeCoercion {
 
         val newLeft =
           if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) {
-            logger.debug(s"Widening numeric types in union $castedLeft ${left.output}")
+            logDebug(s"Widening numeric types in union $castedLeft ${left.output}")
             Project(castedLeft, left)
           } else {
             left
@@ -178,7 +178,7 @@ trait HiveTypeCoercion {
 
         val newRight =
           if (castedRight.map(_.dataType) != right.output.map(_.dataType)) {
-            logger.debug(s"Widening numeric types in union $castedRight ${right.output}")
+            logDebug(s"Widening numeric types in union $castedRight ${right.output}")
             Project(castedRight, right)
           } else {
             right
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index f38f99569f207dca2c20f75e1a42cebb3e0e783c..0913f158887807715a69cf6ea7e13d71a584b19a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.errors.attachTree
 import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.trees
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
index 4211998f7511a8637a96b036e9dd053f237035a5..094ff14552283a5e1d90b480619b0f96ecccbc80 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions.codegen
 
-import com.typesafe.scalalogging.slf4j.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types.{StringType, NumericType}
 
@@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
       }
       new $orderingName()
       """
-    logger.debug(s"Generated Ordering: $code")
+    logDebug(s"Generated Ordering: $code")
     toolBox.eval(code).asInstanceOf[Ordering[Row]]
   }
 }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
index ca9642954eb27144479ef5408aa7a03ccc2b5a13..bdd07bbeb223094bcb8bd2acd55e46d664a1578a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
@@ -25,5 +25,4 @@ package object catalyst {
    */
   protected[catalyst] object ScalaReflectionLock
 
-  protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging
 }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
index 781ba489b44c6d775f20f278c6cc4dec380c715d..5839c9f7c43ef7af25a477a9729505a9d1df3b4e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.planning
 
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.trees.TreeNode
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index bc763a4e06e673271af69baaafdb278c23785078..90923fe31a0637c521d4dadc31760542b2176ac4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning
 import scala.annotation.tailrec
 
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 
@@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
 
   def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
     case join @ Join(left, right, joinType, condition) =>
-      logger.debug(s"Considering join on: $condition")
+      logDebug(s"Considering join on: $condition")
       // Find equi-join predicates that can be evaluated before the join, and thus can be used
       // as join keys.
       val (joinPredicates, otherPredicates) = 
@@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
       val rightKeys = joinKeys.map(_._2)
 
       if (joinKeys.nonEmpty) {
-        logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
+        logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
         Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right))
       } else {
         None
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
index f8960b3fe7a170fe0f30e17cbab9c64396157844..03414b2301e816552fb7001b42873d6d85bc3c3f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.rules
 
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.trees.TreeNode
 
 abstract class Rule[TreeType <: TreeNode[_]] extends Logging {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index 6aa407c836aec4e7b46079ffb295316723ddde6f..d192b151ac1c3116cc984ce15fade5fd4bb64281 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.rules
 
-import org.apache.spark.sql.catalyst.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.trees.TreeNode
 import org.apache.spark.sql.catalyst.util.sideBySide
 
@@ -60,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
           case (plan, rule) =>
             val result = rule(plan)
             if (!result.fastEquals(plan)) {
-              logger.trace(
+              logTrace(
                 s"""
                   |=== Applying Rule ${rule.ruleName} ===
                   |${sideBySide(plan.treeString, result.treeString).mkString("\n")}
@@ -73,26 +73,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
         if (iteration > batch.strategy.maxIterations) {
           // Only log if this is a rule that is supposed to run more than once.
           if (iteration != 2) {
-            logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
+            logInfo(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
           }
           continue = false
         }
 
         if (curPlan.fastEquals(lastPlan)) {
-          logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
+          logTrace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
           continue = false
         }
         lastPlan = curPlan
       }
 
       if (!batchStartPlan.fastEquals(curPlan)) {
-        logger.debug(
+        logDebug(
           s"""
           |=== Result of Batch ${batch.name} ===
           |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")}
         """.stripMargin)
       } else {
-        logger.trace(s"Batch ${batch.name} has no effect.")
+        logTrace(s"Batch ${batch.name} has no effect.")
       }
     }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index 9a28d035a10a3510ea17d354bda10f533b149ad0..d725a92c06f7bf38c02443f87f84b35d3833450a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.catalyst
 
+import org.apache.spark.Logging
+
 /**
  * A library for easily manipulating trees of operators.  Operators that extend TreeNode are
  * granted the following interface:
@@ -31,8 +33,8 @@ package org.apache.spark.sql.catalyst
  *   <li>debugging support - pretty printing, easy splicing of trees, etc.</li>
  * </ul>
  */
-package object trees {
+package object trees extends Logging {
   // Since we want tree nodes to be lightweight, we create one logger for all treenode instances.
-  protected val logger =
-    com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees"))
+  protected override def logName = "catalyst.trees"
+
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index dad71079c29b991c7151d3f1aadb0bb2a9471ff1..00dd34aabc3890b0cc3d61c8231dd16590ce54c4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.execution._
 import org.apache.spark.sql.execution.SparkStrategies
 import org.apache.spark.sql.json._
 import org.apache.spark.sql.parquet.ParquetRelation
-import org.apache.spark.SparkContext
+import org.apache.spark.{Logging, SparkContext}
 
 /**
  * :: AlphaComponent ::
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
index 4c6675c3c87bf0edc25b06447b32484bf5b62e28..6ad12a0dcb64d4fb66f39b196a411a2c08bab2e3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
@@ -19,7 +19,8 @@ package org.apache.spark.sql.columnar.compression
 
 import java.nio.{ByteBuffer, ByteOrder}
 
-import org.apache.spark.sql.{Logging, Row}
+import org.apache.spark.Logging
+import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder}
 
@@ -101,7 +102,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType]
 
     copyColumnHeader(rawBuffer, compressedBuffer)
 
-    logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}")
+    logInfo(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}")
     encoder.compress(rawBuffer, compressedBuffer, columnType)
   }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 30712f03cab4c9754e0b800dcc7794f734259266..77dc2ad733215b59f6098a1522b7c655a8d86273 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -101,7 +101,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl
         !operator.requiredChildDistribution.zip(operator.children).map {
           case (required, child) =>
             val valid = child.outputPartitioning.satisfies(required)
-            logger.debug(
+            logDebug(
               s"${if (valid) "Valid" else "Invalid"} distribution," +
                 s"required: $required current: ${child.outputPartitioning}")
             valid
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index 70db1ebd3a3e1c357444123645e46a63d5e7e6d6..a3d2a1c7a51f818bb4cfda8bbd6ee865a908601d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
 
 private[sql] object JsonRDD extends Logging {
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
index 0995a4eb6299f5c79cd6ad85acef4a79d898903e..f513eae9c2d13bc20d8b246e2b559609ababb264 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
@@ -32,8 +32,6 @@ import org.apache.spark.annotation.DeveloperApi
  */
 package object sql {
 
-  protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging
-
   /**
    * :: DeveloperApi ::
    *
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
index 829342215e69187e96ef2ebcddf4a4a317382b89..75f653f3280bdd9fdef1fd5d5e7ace57b5e3b7e6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -22,7 +22,7 @@ import java.sql.Timestamp
 
 import org.scalatest.FunSuite
 
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.types._
 import org.apache.spark.sql.columnar.ColumnarTestUtils._
 import org.apache.spark.sql.execution.SparkSqlSerializer
@@ -166,7 +166,7 @@ class ColumnTypeSuite extends FunSuite with Logging {
 
       buffer.rewind()
       seq.foreach { expected =>
-        logger.info("buffer = " + buffer + ", expected = " + expected)
+        logInfo("buffer = " + buffer + ", expected = " + expected)
         val extracted = columnType.extract(buffer)
         assert(
           expected === extracted,
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 ddbc2a79fb5128e1df033c8abe78f8cd11857fdf..08d3f983d9e71df99cde60a0950ba39cf132b907 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 cb17d7ce58ea0706b4aff5acaa2fe1a5d2dfd8c2..4d0c506c5a39793d923208c9f9ea88872e55fef2 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 a56b19a4bcda033261b4dea911040e9bfd90b4fb..d362d599d08ca3ebe59a030b7faabcc56c3c9557 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 451c3bd7b9352dc6744c684dd80573d0978cbf44..582264eb59f832084ab15ed8d2e4f3485dce5de0 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 a4e1f3e762e89382acf8f489b89d666efb1483a4..d4dadfd21d13fc9e0a6ddf1719253a827e536944 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 fe3403b3292ecec36bf49ec1b0a7f43fd6844bc2..b7b7c9957ac34ff7c71146774c9d8d87c68d2e6d 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
 
 /**
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 7e3b8727bebed9544a179b1671636391eac7dfd9..2c7270d9f83a9715461bcdeefbea9e16f03613ca 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -207,7 +207,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
       }
     } catch {
       case e: Exception =>
-        logger.error(
+        logError(
           s"""
             |======================
             |HIVE FAILURE OUTPUT
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index fa4e78439c26c47ee2204eb4c501e0b2da71b888..df3604439e4839067f3fa086e5b7fb89f259feb7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -28,7 +28,8 @@ import org.apache.hadoop.hive.ql.plan.TableDesc
 import org.apache.hadoop.hive.serde2.Deserializer
 
 import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.{SQLContext, Logging}
+import org.apache.spark.Logging
+import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog}
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index c50e8c4b5c5d356b8559ae8ff8348b598433f72d..728452a25a00e4c3ce05a783b09578a31a58b515 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -148,7 +148,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
         describedTables ++
         logical.collect { case UnresolvedRelation(databaseName, name, _) => name }
       val referencedTestTables = referencedTables.filter(testTables.contains)
-      logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}")
+      logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}")
       referencedTestTables.foreach(loadTestTable)
       // Proceed with analysis.
       analyzer(logical)
@@ -273,7 +273,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
     if (!(loadedTables contains name)) {
       // Marks the table as loaded first to prevent infite mutually recursive table loading.
       loadedTables += name
-      logger.info(s"Loading test table $name")
+      logInfo(s"Loading test table $name")
       val createCmds =
         testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name"))
       createCmds.foreach(_())
@@ -312,7 +312,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
 
       loadedTables.clear()
       catalog.client.getAllTables("default").foreach { t =>
-        logger.debug(s"Deleting table $t")
+        logDebug(s"Deleting table $t")
         val table = catalog.client.getTable("default", t)
 
         catalog.client.getIndexes("default", t, 255).foreach { index =>
@@ -325,7 +325,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
       }
 
       catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db =>
-        logger.debug(s"Dropping Database: $db")
+        logDebug(s"Dropping Database: $db")
         catalog.client.dropDatabase(db, true, false, true)
       }
 
@@ -347,7 +347,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
       loadTestTable("srcpart")
     } catch {
       case e: Exception =>
-        logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e")
+        logError(s"FATAL ERROR: Failed to reset TestDB state. $e")
         // At this point there is really no reason to continue, but the test framework traps exits.
         // So instead we just pause forever so that at least the developer can see where things
         // started to go wrong.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 7582b4743d404ff15ff57b93fa56334ad52619e7..d181921269b5629173b82c7458a551793982daa9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry}
 import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType}
 import org.apache.hadoop.hive.ql.udf.generic._
 
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.analysis
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types._
@@ -119,7 +119,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[
       sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}."))
 
     (a: Any) => {
-      logger.debug(
+      logDebug(
         s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.")
       // We must make sure that primitives get boxed java style.
       if (a == null) {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 6c8fe4b196deae96251c21aac4ef779e5e73c948..83cfbc6b4a00242ddd6b792c6781e20d37c278bd 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -21,7 +21,7 @@ import java.io._
 
 import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
 
-import org.apache.spark.sql.Logging
+import org.apache.spark.Logging
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand}
@@ -197,7 +197,7 @@ abstract class HiveComparisonTest
     // If test sharding is enable, skip tests that are not in the correct shard.
     shardInfo.foreach {
       case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return
-      case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'")
+      case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'")
     }
 
     // Skip tests found in directories specified by user.
@@ -213,13 +213,13 @@ abstract class HiveComparisonTest
         .map(new File(_, testCaseName))
         .filter(_.exists)
     if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) {
-      logger.debug(
+      logDebug(
         s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}")
       return
     }
 
     test(testCaseName) {
-      logger.debug(s"=== HIVE TEST: $testCaseName ===")
+      logDebug(s"=== HIVE TEST: $testCaseName ===")
 
       // Clear old output for this testcase.
       outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete())
@@ -235,7 +235,7 @@ abstract class HiveComparisonTest
           .filterNot(_ contains "hive.outerjoin.supports.filters")
 
       if (allQueries != queryList)
-        logger.warn(s"Simplifications made on unsupported operations for test $testCaseName")
+        logWarning(s"Simplifications made on unsupported operations for test $testCaseName")
 
       lazy val consoleTestCase = {
         val quotes = "\"\"\""
@@ -257,11 +257,11 @@ abstract class HiveComparisonTest
         }
 
         val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile =>
-          logger.debug(s"Looking for cached answer file $cachedAnswerFile.")
+          logDebug(s"Looking for cached answer file $cachedAnswerFile.")
           if (cachedAnswerFile.exists) {
             Some(fileToString(cachedAnswerFile))
           } else {
-            logger.debug(s"File $cachedAnswerFile not found")
+            logDebug(s"File $cachedAnswerFile not found")
             None
           }
         }.map {
@@ -272,7 +272,7 @@ abstract class HiveComparisonTest
 
         val hiveResults: Seq[Seq[String]] =
           if (hiveCachedResults.size == queryList.size) {
-            logger.info(s"Using answer cache for test: $testCaseName")
+            logInfo(s"Using answer cache for test: $testCaseName")
             hiveCachedResults
           } else {
 
@@ -287,7 +287,7 @@ abstract class HiveComparisonTest
                   if (installHooksCommand.findAllMatchIn(queryString).nonEmpty)
                     sys.error("hive exec hooks not supported for tests.")
 
-                  logger.warn(s"Running query ${i+1}/${queryList.size} with hive.")
+                  logWarning(s"Running query ${i+1}/${queryList.size} with hive.")
                   // Analyze the query with catalyst to ensure test tables are loaded.
                   val answer = hiveQuery.analyzed match {
                     case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output.
@@ -351,7 +351,7 @@ abstract class HiveComparisonTest
               val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n")
 
               if (recomputeCache) {
-                logger.warn(s"Clearing cache files for failed test $testCaseName")
+                logWarning(s"Clearing cache files for failed test $testCaseName")
                 hiveCacheFiles.foreach(_.delete())
               }
 
@@ -380,7 +380,7 @@ abstract class HiveComparisonTest
               TestHive.runSqlHive("SELECT key FROM src")
             } catch {
               case e: Exception =>
-                logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.")
+                logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.")
                 // The testing setup traps exits so wait here for a long time so the developer can see when things started
                 // to go wrong.
                 Thread.sleep(1000000)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
index 50ab71a9003d34b049dd6c9b64704c87fab3cc18..02518d516261bc0fe7b37cd0ba5e4fb712ab2df5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
@@ -53,7 +53,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest {
   testCases.sorted.foreach {
     case (testCaseName, testCaseFile) =>
       if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) {
-        logger.debug(s"Blacklisted test skipped $testCaseName")
+        logDebug(s"Blacklisted test skipped $testCaseName")
       } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) {
         // Build a test case and submit it to scala test framework...
         val queriesString = fileToString(testCaseFile)