Skip to content

Commit 8c321d6

Browse files
committed
[SPARK-8118] [SQL] Mutes noisy Parquet log output reappeared after upgrading Parquet to 1.7.0
Author: Cheng Lian <lian@databricks.com> Closes #6670 from liancheng/spark-8118 and squashes the following commits: b6e85a6 [Cheng Lian] Suppresses unnecesary ParquetRecordReader log message (PARQUET-220) 385603c [Cheng Lian] Mutes noisy Parquet log output reappeared after upgrading Parquet to 1.7.0
1 parent 0ac4708 commit 8c321d6

File tree

1 file changed

+20
-15
lines changed

1 file changed

+20
-15
lines changed

sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala

Lines changed: 20 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -18,20 +18,21 @@
1818
package org.apache.spark.sql.parquet
1919

2020
import java.io.IOException
21-
import java.util.logging.Level
21+
import java.util.logging.{Level, Logger => JLogger}
2222

2323
import org.apache.hadoop.conf.Configuration
2424
import org.apache.hadoop.fs.Path
2525
import org.apache.hadoop.fs.permission.FsAction
26-
import org.apache.spark.sql.types.{StructType, DataType}
27-
import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat}
2826
import org.apache.parquet.hadoop.metadata.CompressionCodecName
27+
import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat, ParquetRecordReader}
2928
import org.apache.parquet.schema.MessageType
29+
import org.apache.parquet.{Log => ParquetLog}
3030

31-
import org.apache.spark.sql.{DataFrame, SQLContext}
3231
import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException}
33-
import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute}
32+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
3433
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
34+
import org.apache.spark.sql.types.StructType
35+
import org.apache.spark.sql.{DataFrame, SQLContext}
3536

3637
/**
3738
* Relation that consists of data stored in a Parquet columnar format.
@@ -94,33 +95,37 @@ private[sql] case class ParquetRelation(
9495
private[sql] object ParquetRelation {
9596

9697
def enableLogForwarding() {
97-
// Note: the parquet.Log class has a static initializer that
98-
// sets the java.util.logging Logger for "parquet". This
98+
// Note: the org.apache.parquet.Log class has a static initializer that
99+
// sets the java.util.logging Logger for "org.apache.parquet". This
99100
// checks first to see if there's any handlers already set
100101
// and if not it creates them. If this method executes prior
101102
// to that class being loaded then:
102103
// 1) there's no handlers installed so there's none to
103104
// remove. But when it IS finally loaded the desired affect
104105
// of removing them is circumvented.
105-
// 2) The parquet.Log static initializer calls setUseParentHanders(false)
106+
// 2) The parquet.Log static initializer calls setUseParentHandlers(false)
106107
// undoing the attempt to override the logging here.
107108
//
108109
// Therefore we need to force the class to be loaded.
109110
// This should really be resolved by Parquet.
110-
Class.forName(classOf[org.apache.parquet.Log].getName)
111+
Class.forName(classOf[ParquetLog].getName)
111112

112113
// Note: Logger.getLogger("parquet") has a default logger
113114
// that appends to Console which needs to be cleared.
114-
val parquetLogger = java.util.logging.Logger.getLogger("parquet")
115+
val parquetLogger = JLogger.getLogger(classOf[ParquetLog].getPackage.getName)
115116
parquetLogger.getHandlers.foreach(parquetLogger.removeHandler)
116-
// TODO(witgo): Need to set the log level ?
117-
// if(parquetLogger.getLevel != null) parquetLogger.setLevel(null)
118-
if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true)
117+
parquetLogger.setUseParentHandlers(true)
119118

120-
// Disables WARN log message in ParquetOutputCommitter.
119+
// Disables a WARN log message in ParquetOutputCommitter. We first ensure that
120+
// ParquetOutputCommitter is loaded and the static LOG field gets initialized.
121121
// See https://issues.apache.org/jira/browse/SPARK-5968 for details
122122
Class.forName(classOf[ParquetOutputCommitter].getName)
123-
java.util.logging.Logger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
123+
JLogger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF)
124+
125+
// Similar as above, disables a unnecessary WARN log message in ParquetRecordReader.
126+
// See https://issues.apache.org/jira/browse/PARQUET-220 for details
127+
Class.forName(classOf[ParquetRecordReader[_]].getName)
128+
JLogger.getLogger(classOf[ParquetRecordReader[_]].getName).setLevel(Level.OFF)
124129
}
125130

126131
// The element type for the RDDs that this relation maps to.

0 commit comments

Comments
 (0)