Skip to content

Commit d20ecaf

Browse files
HyukjinKwonRobert Kruszewski
authored andcommitted
[SPARK-24967][SQL] Avro: Use internal.Logging instead for logging
## What changes were proposed in this pull request? Looks Avro uses direct `getLogger` to create a SLF4J logger. Should better use `internal.Logging` instead. ## How was this patch tested? Exiting tests. Author: hyukjinkwon <[email protected]> Closes apache#21914 from HyukjinKwon/avro-log.
1 parent 9475d94 commit d20ecaf

File tree

1 file changed

+8
-10
lines changed

1 file changed

+8
-10
lines changed

external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala

Lines changed: 8 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package org.apache.spark.sql.avro
1919

2020
import java.io._
2121
import java.net.URI
22-
import java.util.zip.Deflater
2322

2423
import scala.util.control.NonFatal
2524

@@ -31,18 +30,18 @@ import org.apache.avro.mapreduce.AvroJob
3130
import org.apache.hadoop.conf.Configuration
3231
import org.apache.hadoop.fs.{FileStatus, Path}
3332
import org.apache.hadoop.mapreduce.Job
34-
import org.slf4j.LoggerFactory
3533

3634
import org.apache.spark.TaskContext
35+
import org.apache.spark.internal.Logging
3736
import org.apache.spark.sql.SparkSession
3837
import org.apache.spark.sql.catalyst.InternalRow
3938
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile}
4039
import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
4140
import org.apache.spark.sql.types.StructType
4241
import org.apache.spark.util.SerializableConfiguration
4342

44-
private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
45-
private val log = LoggerFactory.getLogger(getClass)
43+
private[avro] class AvroFileFormat extends FileFormat
44+
with DataSourceRegister with Logging with Serializable {
4645

4746
override def equals(other: Any): Boolean = other match {
4847
case _: AvroFileFormat => true
@@ -121,23 +120,23 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
121120

122121
parsedOptions.compression match {
123122
case "uncompressed" =>
124-
log.info("writing uncompressed Avro records")
123+
logInfo("writing uncompressed Avro records")
125124
job.getConfiguration.setBoolean(COMPRESS_KEY, false)
126125

127126
case "snappy" =>
128-
log.info("compressing Avro output using Snappy")
127+
logInfo("compressing Avro output using Snappy")
129128
job.getConfiguration.setBoolean(COMPRESS_KEY, true)
130129
job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, DataFileConstants.SNAPPY_CODEC)
131130

132131
case "deflate" =>
133132
val deflateLevel = spark.sessionState.conf.avroDeflateLevel
134-
log.info(s"compressing Avro output using deflate (level=$deflateLevel)")
133+
logInfo(s"compressing Avro output using deflate (level=$deflateLevel)")
135134
job.getConfiguration.setBoolean(COMPRESS_KEY, true)
136135
job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, DataFileConstants.DEFLATE_CODEC)
137136
job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel)
138137

139138
case unknown: String =>
140-
log.error(s"unsupported compression codec $unknown")
139+
logError(s"unsupported compression codec $unknown")
141140
}
142141

143142
new AvroOutputWriterFactory(dataSchema, outputAvroSchema.toString)
@@ -157,7 +156,6 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
157156
val parsedOptions = new AvroOptions(options, hadoopConf)
158157

159158
(file: PartitionedFile) => {
160-
val log = LoggerFactory.getLogger(classOf[AvroFileFormat])
161159
val conf = broadcastedConf.value.value
162160
val userProvidedSchema = parsedOptions.schema.map(new Schema.Parser().parse)
163161

@@ -176,7 +174,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
176174
DataFileReader.openReader(in, datumReader)
177175
} catch {
178176
case NonFatal(e) =>
179-
log.error("Exception while opening DataFileReader", e)
177+
logError("Exception while opening DataFileReader", e)
180178
in.close()
181179
throw e
182180
}

0 commit comments

Comments
 (0)