diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index e5e15617acb10766055fad27fcd4cc039c72d982..9d429dceeb858c12de209ba37443bf094dfee1f3 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi
  * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
  * logging messages at different levels using methods that only evaluate parameters lazily if the
  * log level is enabled.
- * 
+ *
  * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility.
  *       This will likely be changed or removed in future releases.
  */
@@ -60,7 +60,7 @@ trait Logging {
   protected def logDebug(msg: => String) {
     if (log.isDebugEnabled) log.debug(msg)
   }
-  
+
   protected def logTrace(msg: => String) {
     if (log.isTraceEnabled) log.trace(msg)
   }
@@ -117,10 +117,10 @@ trait Logging {
       val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
       val classLoader = this.getClass.getClassLoader
       Option(classLoader.getResource(defaultLogProps)) match {
-        case Some(url) => 
+        case Some(url) =>
           PropertyConfigurator.configure(url)
           log.info(s"Using Spark's default log4j profile: $defaultLogProps")
-        case None => 
+        case None =>
           System.err.println(s"Spark was unable to load $defaultLogProps")
       }
     }
@@ -135,4 +135,16 @@ trait Logging {
 private object Logging {
   @volatile private var initialized = false
   val initLock = new Object()
+  try {
+    // We use reflection here to handle the case where users remove the
+    // slf4j-to-jul bridge order to route their logs to JUL.
+    val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
+    bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null)
+    val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean]
+    if (!installed) {
+      bridgeClass.getMethod("install").invoke(null)
+    }
+  } catch {
+    case e: ClassNotFoundException => // can't log anything yet so just fail silently
+  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 505ad0a2c77c1028cb5a8639de981453e5f05369..4d7c86a3a4fc71d1fba9f7e846d976a1ad7dd277 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -82,30 +82,13 @@ private[sql] case class ParquetRelation(val path: String)
 private[sql] object ParquetRelation {
 
   def enableLogForwarding() {
-    // Note: Parquet does not use forwarding to parent loggers which
-    // is required for the JUL-SLF4J bridge to work. Also there is
-    // a default logger that appends to Console which needs to be
-    // reset.
-    import org.slf4j.bridge.SLF4JBridgeHandler
-    import java.util.logging.Logger
-    import java.util.logging.LogManager
-
-    val loggerNames = Seq(
-      "parquet.hadoop.ColumnChunkPageWriteStore",
-      "parquet.hadoop.InternalParquetRecordWriter",
-      "parquet.hadoop.ParquetRecordReader",
-      "parquet.hadoop.ParquetInputFormat",
-      "parquet.hadoop.ParquetOutputFormat",
-      "parquet.hadoop.ParquetFileReader",
-      "parquet.hadoop.InternalParquetRecordReader",
-      "parquet.hadoop.codec.CodecConfig")
-    LogManager.getLogManager.reset()
-    SLF4JBridgeHandler.install()
-    for(name <- loggerNames) {
-      val logger = Logger.getLogger(name)
-      logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME))
-      logger.setUseParentHandlers(true)
-    }
+    // Note: Logger.getLogger("parquet") has a default logger
+    // that appends to Console which needs to be cleared.
+    val parquetLogger = java.util.logging.Logger.getLogger("parquet")
+    parquetLogger.getHandlers.foreach(parquetLogger.removeHandler)
+    // TODO(witgo): Need to set the log level ?
+    // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null)
+    if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true)
   }
 
   // The element type for the RDDs that this relation maps to.