From 840333133396d443e747f62fce9967f7681fb276 Mon Sep 17 00:00:00 2001
From: Cheng Lian <lian@databricks.com>
Date: Tue, 24 Feb 2015 10:45:38 -0800
Subject: [PATCH] [SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN
 logs

Please refer to the [JIRA ticket] [1] for the motivation.

[1]: https://issues.apache.org/jira/browse/SPARK-5968

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4744)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #4744 from liancheng/spark-5968 and squashes the following commits:

caac6a8 [Cheng Lian] Suppresses ParquetOutputCommitter WARN logs
---
 .../apache/spark/sql/parquet/ParquetRelation.scala   | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)

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 b0db9943a5..a0d1005c0c 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
@@ -18,11 +18,12 @@
 package org.apache.spark.sql.parquet
 
 import java.io.IOException
+import java.util.logging.Level
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.fs.permission.FsAction
-import parquet.hadoop.ParquetOutputFormat
+import parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat}
 import parquet.hadoop.metadata.CompressionCodecName
 import parquet.schema.MessageType
 
@@ -91,7 +92,7 @@ private[sql] object ParquetRelation {
     // checks first to see if there's any handlers already set
     // and if not it creates them. If this method executes prior
     // to that class being loaded then:
-    //  1) there's no handlers installed so there's none to 
+    //  1) there's no handlers installed so there's none to
     // remove. But when it IS finally loaded the desired affect
     // of removing them is circumvented.
     //  2) The parquet.Log static initializer calls setUseParentHanders(false)
@@ -99,7 +100,7 @@ private[sql] object ParquetRelation {
     //
     // Therefore we need to force the class to be loaded.
     // This should really be resolved by Parquet.
-    Class.forName(classOf[parquet.Log].getName())
+    Class.forName(classOf[parquet.Log].getName)
 
     // Note: Logger.getLogger("parquet") has a default logger
     // that appends to Console which needs to be cleared.
@@ -108,6 +109,11 @@ private[sql] object ParquetRelation {
     // TODO(witgo): Need to set the log level ?
     // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null)
     if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true)
+
+    // Disables WARN log message in ParquetOutputCommitter.
+    // See https://issues.apache.org/jira/browse/SPARK-5968 for details
+    Class.forName(classOf[ParquetOutputCommitter].getName)
+    java.util.logging.Logger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
   }
 
   // The element type for the RDDs that this relation maps to.
-- 
GitLab