diff --git a/core/pom.xml b/core/pom.xml
index 47766ae5fbb3d8c3b9c2e225977fb3ff277fa86a..7c60cf10c3dc25b68e942fc258af875e6c5a39ee 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -98,10 +98,6 @@
       <groupId>org.slf4j</groupId>
       <artifactId>jcl-over-slf4j</artifactId>
     </dependency>
-    <dependency>
-      <groupId>com.typesafe.scala-logging</groupId>
-      <artifactId>scala-logging-slf4j_${scala.binary.version}</artifactId>
-    </dependency>
     <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 6e61c00b8dbbf649ab9213f058a179e02402c767..807ef3e9c9d60376d362a24220d66d92c2d6c814 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -18,9 +18,8 @@
 package org.apache.spark
 
 import org.apache.log4j.{LogManager, PropertyConfigurator}
-import org.slf4j.LoggerFactory
+import org.slf4j.{Logger, LoggerFactory}
 import org.slf4j.impl.StaticLoggerBinder
-import com.typesafe.scalalogging.slf4j.Logger
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.util.Utils
@@ -40,69 +39,61 @@ 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 = {
-    var className = this.getClass.getName
-    // Ignore trailing $'s in the class names for Scala objects
-    if (className.endsWith("$")) {
-      className = className.substring(0, className.length - 1)
-    }
-    className
-  }
-
   // Method to get or create the logger for this object
   protected def log: Logger = {
     if (log_ == null) {
       initializeIfNecessary()
-      log_ = Logger(LoggerFactory.getLogger(logName))
+      var className = this.getClass.getName
+      // Ignore trailing $'s in the class names for Scala objects
+      log_ = LoggerFactory.getLogger(className.stripSuffix("$"))
     }
     log_
   }
 
   // Log methods that take only a String
   protected def logInfo(msg: => String) {
-    log.info(msg)
+    if (log.isInfoEnabled) log.info(msg)
   }
 
   protected def logDebug(msg: => String) {
-    log.debug(msg)
+    if (log.isDebugEnabled) log.debug(msg)
   }
 
   protected def logTrace(msg: => String) {
-    log.trace(msg)
+    if (log.isTraceEnabled) log.trace(msg)
   }
 
   protected def logWarning(msg: => String) {
-    log.warn(msg)
+    if (log.isWarnEnabled) log.warn(msg)
   }
 
   protected def logError(msg: => String) {
-    log.error(msg)
+    if (log.isErrorEnabled) log.error(msg)
   }
 
   // Log methods that take Throwables (Exceptions/Errors) too
   protected def logInfo(msg: => String, throwable: Throwable) {
-    log.info(msg, throwable)
+    if (log.isInfoEnabled) log.info(msg, throwable)
   }
 
   protected def logDebug(msg: => String, throwable: Throwable) {
-    log.debug(msg, throwable)
+    if (log.isDebugEnabled) log.debug(msg, throwable)
   }
 
   protected def logTrace(msg: => String, throwable: Throwable) {
-    log.trace(msg, throwable)
+    if (log.isTraceEnabled) log.trace(msg, throwable)
   }
 
   protected def logWarning(msg: => String, throwable: Throwable) {
-    log.warn(msg, throwable)
+    if (log.isWarnEnabled) log.warn(msg, throwable)
   }
 
   protected def logError(msg: => String, throwable: Throwable) {
-    log.error(msg, throwable)
+    if (log.isErrorEnabled) log.error(msg, throwable)
   }
 
   protected def isTraceEnabled(): Boolean = {
-    log.underlying.isTraceEnabled
+    log.isTraceEnabled
   }
 
   private def initializeIfNecessary() {
diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala
index e84a6b951f65e4170146dd0afb048b1d72890e52..f77488ef3d4494136584ceee15f1919c7a20a9b6 100644
--- a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala
+++ b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.util
 
 import org.apache.commons.lang3.SystemUtils
-import com.typesafe.scalalogging.slf4j.Logger
+import org.slf4j.Logger
 import sun.misc.{Signal, SignalHandler}
 
 /**
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 3007681a44f1c1c9f02afe20fe9644f83a81dd1e..9a33bd1cf6ad1b2529be8b3efb2be0224907629c 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -59,10 +59,6 @@
       <artifactId>breeze_${scala.binary.version}</artifactId>
       <version>0.7</version>
       <exclusions>
-        <exclusion>
-          <groupId>com.typesafe</groupId>
-          <artifactId>scalalogging-slf4j_${scala.binary.version}</artifactId>
-        </exclusion>  
         <!-- This is included as a compile-scoped dependency by jtransforms, which is
              a dependency of breeze. -->
         <exclusion>
diff --git a/pom.xml b/pom.xml
index 9d62cea68995fa68ea810089f2d37bd614beb16f..ae97bf03c53a2ea0790ec205f5b1f766b5498be7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -279,11 +279,6 @@
         <artifactId>slf4j-log4j12</artifactId>
         <version>${slf4j.version}</version>
       </dependency>
-      <dependency>
-        <groupId>com.typesafe.scala-logging</groupId>
-        <artifactId>scala-logging-slf4j_${scala.binary.version}</artifactId>
-        <version>2.1.2</version>
-      </dependency>
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>jul-to-slf4j</artifactId>
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index a0cee1d765c7fc5a9ead2e8534d05b4843114ea8..537ca0dcf267d51269ae8f9fff9fecf5fccb7adb 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -103,101 +103,14 @@ object MimaExcludes {
             ProblemFilters.exclude[IncompatibleMethTypeProblem](
               "org.apache.spark.mllib.tree.impurity.Variance.calculate")
           ) ++
-          Seq( // Package-private classes removed in SPARK-2341
+          Seq ( // Package-private classes removed in SPARK-2341
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"),
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"),
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"),
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"),
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"),
             ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$")
-          ) ++
-           Seq(
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.bagel.Bagel.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.StreamingContext.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.dstream.DStream.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.mllib.recommendation.ALS.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.mllib.clustering.KMeans.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.mllib.classification.NaiveBayes.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.kafka.KafkaReceiver.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.SparkContext.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.rdd.PairRDDFunctions.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.rdd.OrderedRDDFunctions.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.rdd.SequenceFileRDDFunctions.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.rdd.DoubleRDDFunctions.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.twitter.TwitterReceiver.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.flume.FlumeReceiver.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.rdd.RDD.log"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.SparkConf.log"),
-
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.SparkConf.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.StreamingContext.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.dstream.DStream.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.mllib.recommendation.ALS.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.mllib.clustering.KMeans.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.mllib.classification.NaiveBayes.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.SparkContext.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.rdd.RDD.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.rdd.SequenceFileRDDFunctions.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.rdd.OrderedRDDFunctions.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.rdd.DoubleRDDFunctions.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log__="),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleMethTypeProblem]
-               ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log_"),
-             ProblemFilters.exclude[IncompatibleResultTypeProblem]
-               ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_")
-           )
+          )
         case v if v.startsWith("1.0") =>
           Seq(
             MimaBuild.excludeSparkPackage("api.java"),
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 58d44e7923bee027212cf33f4d309b0870cf1d6c..54fa96baa1e1833c01dbe612c8cd8cce492fb531 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -54,6 +54,11 @@
       <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 2b36582215f243b7bbf5667d58496f7376a70cf6..74c0104e5b17f83b84aa8edc9e3e938f9fe32841 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 =>
-        log.trace(s"Attempting to resolve ${q.simpleString}")
+        logger.trace(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)
-            log.debug(s"Resolving $u to $result")
+            logger.debug(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 eafbb70dc3fdd482629a07d936107df4e82bbc98..47c7ad076ad076aeeb127fca812a21968357afce 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) =>
-              log.debug(s"Promoting $a to $newType in ${q.simpleString}}")
+              logger.debug(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 =>
-            log.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
+            logger.debug(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)) {
-            log.debug(s"Widening numeric types in union $castedLeft ${left.output}")
+            logger.debug(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)) {
-            log.debug(s"Widening numeric types in union $castedRight ${right.output}")
+            logger.debug(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 0913f158887807715a69cf6ea7e13d71a584b19a..f38f99569f207dca2c20f75e1a42cebb3e0e783c 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.Logging
+import org.apache.spark.sql.catalyst.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 e2552d432cb715dc2b67edff5a1e1d51bfa1620e..4211998f7511a8637a96b036e9dd053f237035a5 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 org.apache.spark.Logging
+import com.typesafe.scalalogging.slf4j.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()
       """
-    log.debug(s"Generated Ordering: $code")
+    logger.debug(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 bdd07bbeb223094bcb8bd2acd55e46d664a1578a..ca9642954eb27144479ef5408aa7a03ccc2b5a13 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,4 +25,5 @@ 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 5839c9f7c43ef7af25a477a9729505a9d1df3b4e..781ba489b44c6d775f20f278c6cc4dec380c715d 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.Logging
+import org.apache.spark.sql.catalyst.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 06c5ffe92abc871582012fc285d041711fd29fb9..bc763a4e06e673271af69baaafdb278c23785078 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.Logging
+import org.apache.spark.sql.catalyst.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) =>
-      log.debug(s"Considering join on: $condition")
+      logger.debug(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) {
-        log.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
+        logger.debug(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 03414b2301e816552fb7001b42873d6d85bc3c3f..f8960b3fe7a170fe0f30e17cbab9c64396157844 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.Logging
+import org.apache.spark.sql.catalyst.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 20bf8eed7ddf3fc86f8aa9b6d0ccf4995b590672..6aa407c836aec4e7b46079ffb295316723ddde6f 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.Logging
+import org.apache.spark.sql.catalyst.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)) {
-              log.trace(
+              logger.trace(
                 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) {
-            log.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
+            logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}")
           }
           continue = false
         }
 
         if (curPlan.fastEquals(lastPlan)) {
-          log.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
+          logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
           continue = false
         }
         lastPlan = curPlan
       }
 
       if (!batchStartPlan.fastEquals(curPlan)) {
-        log.debug(
+        logger.debug(
           s"""
           |=== Result of Batch ${batch.name} ===
           |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")}
         """.stripMargin)
       } else {
-        log.trace(s"Batch ${batch.name} has no effect.")
+        logger.trace(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 d725a92c06f7bf38c02443f87f84b35d3833450a..9a28d035a10a3510ea17d354bda10f533b149ad0 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,8 +17,6 @@
 
 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:
@@ -33,8 +31,8 @@ import org.apache.spark.Logging
  *   <li>debugging support - pretty printing, easy splicing of trees, etc.</li>
  * </ul>
  */
-package object trees extends Logging {
+package object trees {
   // Since we want tree nodes to be lightweight, we create one logger for all treenode instances.
-  protected override def logName = "catalyst.trees"
-
+  protected val logger =
+    com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("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 00dd34aabc3890b0cc3d61c8231dd16590ce54c4..dad71079c29b991c7151d3f1aadb0bb2a9471ff1 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.{Logging, SparkContext}
+import org.apache.spark.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 828a8896ff60a2188d6976b738ca7c4e4be70b74..4c6675c3c87bf0edc25b06447b32484bf5b62e28 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,8 +19,7 @@ package org.apache.spark.sql.columnar.compression
 
 import java.nio.{ByteBuffer, ByteOrder}
 
-import org.apache.spark.Logging
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{Logging, Row}
 import org.apache.spark.sql.catalyst.types.NativeType
 import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder}
 
@@ -102,7 +101,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType]
 
     copyColumnHeader(rawBuffer, compressedBuffer)
 
-    log.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}")
+    logger.info(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 0c3d537ccb494bc46c6bd858902fae8769de5376..30712f03cab4c9754e0b800dcc7794f734259266 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)
-            log.debug(
+            logger.debug(
               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 a3d2a1c7a51f818bb4cfda8bbd6ee865a908601d..70db1ebd3a3e1c357444123645e46a63d5e7e6d6 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.Logging
+import org.apache.spark.sql.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 f513eae9c2d13bc20d8b246e2b559609ababb264..0995a4eb6299f5c79cd6ad85acef4a79d898903e 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,6 +32,8 @@ 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 a165531573a208d3d7212757b4c5b945ed1ac217..829342215e69187e96ef2ebcddf4a4a317382b89 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.Logging
+import org.apache.spark.sql.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 =>
-        log.info("buffer = " + buffer + ", expected = " + expected)
+        logger.info("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 5959ba3d23f8e300b66e8fcfb21eff0654e5ae3e..ddbc2a79fb5128e1df033c8abe78f8cd11857fdf 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 4d0c506c5a39793d923208c9f9ea88872e55fef2..cb17d7ce58ea0706b4aff5acaa2fe1a5d2dfd8c2 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 276723990b2addd63077750c09fc9353359fccc4..a56b19a4bcda033261b4dea911040e9bfd90b4fb 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 dfc93b19d019c396a55a909e2d7669f889072bd2..451c3bd7b9352dc6744c684dd80573d0978cbf44 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 2c6e24e80d6dd03db69944bde3376841dc729274..a4e1f3e762e89382acf8f489b89d666efb1483a4 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 b7b7c9957ac34ff7c71146774c9d8d87c68d2e6d..fe3403b3292ecec36bf49ec1b0a7f43fd6844bc2 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
 
 /**
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 1f31d35eaa10d75721b287f509bc4833fc3815ec..7e3b8727bebed9544a179b1671636391eac7dfd9 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 =>
-        log.error(
+        logger.error(
           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 df3604439e4839067f3fa086e5b7fb89f259feb7..fa4e78439c26c47ee2204eb4c501e0b2da71b888 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,8 +28,7 @@ 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.Logging
-import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.{SQLContext, Logging}
 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 7376fb5dc83f82b9ccd30cb98bd2c0f89a5e4902..c50e8c4b5c5d356b8559ae8ff8348b598433f72d 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)
-      log.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}")
+      logger.debug(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
-      log.info(s"Loading test table $name")
+      logger.info(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 =>
-        log.debug(s"Deleting table $t")
+        logger.debug(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 =>
-        log.debug(s"Dropping Database: $db")
+        logger.debug(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 =>
-        log.error(s"FATAL ERROR: Failed to reset TestDB state. $e")
+        logger.error(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 4d8eaa18d784432b303a99bdfd11401e19501419..7582b4743d404ff15ff57b93fa56334ad52619e7 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.Logging
+import org.apache.spark.sql.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) => {
-      log.debug(
+      logger.debug(
         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 52cb1cf986f169cea657a4cedf7c6fe108b7c584..6c8fe4b196deae96251c21aac4ef779e5e73c948 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.Logging
+import org.apache.spark.sql.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, _) => log.debug(s"Shard $shardId includes test '$testCaseName'")
+      case (shardId, _) => logger.debug(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) {
-      log.debug(
+      logger.debug(
         s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}")
       return
     }
 
     test(testCaseName) {
-      log.debug(s"=== HIVE TEST: $testCaseName ===")
+      logger.debug(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)
-        log.warn(s"Simplifications made on unsupported operations for test $testCaseName")
+        logger.warn(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 =>
-          log.debug(s"Looking for cached answer file $cachedAnswerFile.")
+          logger.debug(s"Looking for cached answer file $cachedAnswerFile.")
           if (cachedAnswerFile.exists) {
             Some(fileToString(cachedAnswerFile))
           } else {
-            log.debug(s"File $cachedAnswerFile not found")
+            logger.debug(s"File $cachedAnswerFile not found")
             None
           }
         }.map {
@@ -272,7 +272,7 @@ abstract class HiveComparisonTest
 
         val hiveResults: Seq[Seq[String]] =
           if (hiveCachedResults.size == queryList.size) {
-            log.info(s"Using answer cache for test: $testCaseName")
+            logger.info(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.")
 
-                  log.warn(s"Running query ${i+1}/${queryList.size} with hive.")
+                  logger.warn(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) {
-                log.warn(s"Clearing cache files for failed test $testCaseName")
+                logger.warn(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 =>
-                log.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.")
+                logger.error(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 9ca5575c1be8a04578452692bf9a3bbcf1400fb1..50ab71a9003d34b049dd6c9b64704c87fab3cc18 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(_||_)) {
-        log.debug(s"Blacklisted test skipped $testCaseName")
+        logger.debug(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)