|
18 | 18 | package org.apache.spark.sql.parquet |
19 | 19 |
|
20 | 20 | import java.io.IOException |
21 | | -import java.util.logging.Level |
| 21 | +import java.util.logging.{Level, Logger => JLogger} |
22 | 22 |
|
23 | 23 | import org.apache.hadoop.conf.Configuration |
24 | 24 | import org.apache.hadoop.fs.Path |
25 | 25 | import org.apache.hadoop.fs.permission.FsAction |
26 | | -import org.apache.spark.sql.types.{StructType, DataType} |
27 | | -import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat} |
28 | 26 | import org.apache.parquet.hadoop.metadata.CompressionCodecName |
| 27 | +import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat, ParquetRecordReader} |
29 | 28 | import org.apache.parquet.schema.MessageType |
| 29 | +import org.apache.parquet.{Log => ParquetLog} |
30 | 30 |
|
31 | | -import org.apache.spark.sql.{DataFrame, SQLContext} |
32 | 31 | 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} |
34 | 33 | 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} |
35 | 36 |
|
36 | 37 | /** |
37 | 38 | * Relation that consists of data stored in a Parquet columnar format. |
@@ -94,33 +95,37 @@ private[sql] case class ParquetRelation( |
94 | 95 | private[sql] object ParquetRelation { |
95 | 96 |
|
96 | 97 | 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 |
99 | 100 | // checks first to see if there's any handlers already set |
100 | 101 | // and if not it creates them. If this method executes prior |
101 | 102 | // to that class being loaded then: |
102 | 103 | // 1) there's no handlers installed so there's none to |
103 | 104 | // remove. But when it IS finally loaded the desired affect |
104 | 105 | // 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) |
106 | 107 | // undoing the attempt to override the logging here. |
107 | 108 | // |
108 | 109 | // Therefore we need to force the class to be loaded. |
109 | 110 | // This should really be resolved by Parquet. |
110 | | - Class.forName(classOf[org.apache.parquet.Log].getName) |
| 111 | + Class.forName(classOf[ParquetLog].getName) |
111 | 112 |
|
112 | 113 | // Note: Logger.getLogger("parquet") has a default logger |
113 | 114 | // 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) |
115 | 116 | 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) |
119 | 118 |
|
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. |
121 | 121 | // See https://issues.apache.org/jira/browse/SPARK-5968 for details |
122 | 122 | 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) |
124 | 129 | } |
125 | 130 |
|
126 | 131 | // The element type for the RDDs that this relation maps to. |
|
0 commit comments