/
OrcFileFormat.scala
380 lines (322 loc) · 13.5 KB
/
OrcFileFormat.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.hive.orc
import java.net.URI
import java.nio.charset.StandardCharsets.UTF_8
import java.util.Properties
import scala.collection.JavaConverters._
import scala.util.control.NonFatal
import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.io.Output
import org.apache.commons.codec.binary.Base64
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.ql.io.orc._
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument
import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector}
import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils}
import org.apache.hadoop.io.{NullWritable, Writable}
import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter}
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.orc.OrcConf.COMPRESS
import org.apache.spark.{SPARK_VERSION_SHORT, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.orc.{OrcFilters, OrcOptions}
import org.apache.spark.sql.hive.{HiveInspectors, HiveShim}
import org.apache.spark.sql.sources.{Filter, _}
import org.apache.spark.sql.types._
import org.apache.spark.util.SerializableConfiguration
/**
* `FileFormat` for reading ORC files. If this is moved or renamed, please update
* `DataSource`'s backwardCompatibilityMap.
*/
class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable {
override def shortName(): String = "orc"
override def toString: String = "ORC"
override def inferSchema(
sparkSession: SparkSession,
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = {
val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf)
if (orcOptions.mergeSchema) {
SchemaMergeUtils.mergeSchemasInParallel(
sparkSession, options, files, OrcFileOperator.readOrcSchemasInParallel)
} else {
val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles
OrcFileOperator.readSchema(
files.map(_.getPath.toString),
Some(sparkSession.sessionState.newHadoopConfWithOptions(options)),
ignoreCorruptFiles
)
}
}
override def prepareWrite(
sparkSession: SparkSession,
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf)
val configuration = job.getConfiguration
configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec)
configuration match {
case conf: JobConf =>
conf.setOutputFormat(classOf[OrcOutputFormat])
case conf =>
conf.setClass(
"mapred.output.format.class",
classOf[OrcOutputFormat],
classOf[MapRedOutputFormat[_, _]])
}
new OutputWriterFactory {
override def newInstance(
path: String,
dataSchema: StructType,
context: TaskAttemptContext): OutputWriter = {
new OrcOutputWriter(path, dataSchema, context)
}
override def getFileExtension(context: TaskAttemptContext): String = {
val compressionExtension: String = {
val name = context.getConfiguration.get(COMPRESS.getAttribute)
OrcFileFormat.extensionsForCompressionCodecNames.getOrElse(name, "")
}
compressionExtension + ".orc"
}
}
}
override def isSplitable(
sparkSession: SparkSession,
options: Map[String, String],
path: Path): Boolean = {
true
}
override def buildReader(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
if (sparkSession.sessionState.conf.orcFilterPushDown) {
// Sets pushed predicates
OrcFilters.createFilter(requiredSchema, filters).foreach { f =>
hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, toKryo(f))
hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true)
}
}
val broadcastedHadoopConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles
(file: PartitionedFile) => {
val conf = broadcastedHadoopConf.value.value
val filePath = new Path(new URI(file.filePath))
// SPARK-8501: Empty ORC files always have an empty schema stored in their footer. In this
// case, `OrcFileOperator.readSchema` returns `None`, and we can't read the underlying file
// using the given physical schema. Instead, we simply return an empty iterator.
val isEmptyFile =
OrcFileOperator.readSchema(Seq(filePath.toString), Some(conf), ignoreCorruptFiles).isEmpty
if (isEmptyFile) {
Iterator.empty
} else {
OrcFileFormat.setRequiredColumns(conf, dataSchema, requiredSchema)
val orcRecordReader = {
val job = Job.getInstance(conf)
FileInputFormat.setInputPaths(job, file.filePath)
// Custom OrcRecordReader is used to get
// ObjectInspector during recordReader creation itself and can
// avoid NameNode call in unwrapOrcStructs per file.
// Specifically would be helpful for partitioned datasets.
val orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf))
new SparkOrcNewRecordReader(orcReader, conf, file.start, file.length)
}
val recordsIterator = new RecordReaderIterator[OrcStruct](orcRecordReader)
Option(TaskContext.get())
.foreach(_.addTaskCompletionListener[Unit](_ => recordsIterator.close()))
// Unwraps `OrcStruct`s to `UnsafeRow`s
OrcFileFormat.unwrapOrcStructs(
conf,
dataSchema,
requiredSchema,
Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]),
recordsIterator)
}
}
}
override def supportDataType(dataType: DataType): Boolean = dataType match {
case _: AtomicType => true
case st: StructType => st.forall { f => supportDataType(f.dataType) }
case ArrayType(elementType, _) => supportDataType(elementType)
case MapType(keyType, valueType, _) =>
supportDataType(keyType) && supportDataType(valueType)
case udt: UserDefinedType[_] => supportDataType(udt.sqlType)
case _ => false
}
// HIVE-11253 moved `toKryo` from `SearchArgument` to `storage-api` module.
// This is copied from Hive 1.2's SearchArgumentImpl.toKryo().
private def toKryo(sarg: SearchArgument): String = {
val kryo = new Kryo()
val out = new Output(4 * 1024, 10 * 1024 * 1024)
kryo.writeObject(out, sarg)
out.close()
Base64.encodeBase64String(out.toBytes)
}
}
private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration)
extends HiveInspectors {
def serialize(row: InternalRow): Writable = {
wrapOrcStruct(cachedOrcStruct, structOI, row)
serializer.serialize(cachedOrcStruct, structOI)
}
private[this] val serializer = {
val table = new Properties()
table.setProperty("columns", dataSchema.fieldNames.mkString(","))
table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":"))
val serde = new OrcSerde
serde.initialize(conf, table)
serde
}
// Object inspector converted from the schema of the relation to be serialized.
private[this] val structOI = {
val typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(dataSchema.catalogString)
OrcStruct.createObjectInspector(typeInfo.asInstanceOf[StructTypeInfo])
.asInstanceOf[SettableStructObjectInspector]
}
private[this] val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct]
// Wrapper functions used to wrap Spark SQL input arguments into Hive specific format
private[this] val wrappers = dataSchema.zip(structOI.getAllStructFieldRefs().asScala.toSeq).map {
case (f, i) => wrapperFor(i.getFieldObjectInspector, f.dataType)
}
private[this] def wrapOrcStruct(
struct: OrcStruct,
oi: SettableStructObjectInspector,
row: InternalRow): Unit = {
val fieldRefs = oi.getAllStructFieldRefs
var i = 0
val size = fieldRefs.size
while (i < size) {
oi.setStructFieldData(
struct,
fieldRefs.get(i),
wrappers(i)(row.get(i, dataSchema(i).dataType))
)
i += 1
}
}
}
private[orc] class OrcOutputWriter(
path: String,
dataSchema: StructType,
context: TaskAttemptContext)
extends OutputWriter {
private[this] val serializer = new OrcSerializer(dataSchema, context.getConfiguration)
// `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this
// flag to decide whether `OrcRecordWriter.close()` needs to be called.
private var recordWriterInstantiated = false
private lazy val recordWriter: RecordWriter[NullWritable, Writable] = {
recordWriterInstantiated = true
new OrcOutputFormat().getRecordWriter(
new Path(path).getFileSystem(context.getConfiguration),
context.getConfiguration.asInstanceOf[JobConf],
path,
Reporter.NULL
).asInstanceOf[RecordWriter[NullWritable, Writable]]
}
override def write(row: InternalRow): Unit = {
recordWriter.write(NullWritable.get(), serializer.serialize(row))
}
override def close(): Unit = {
if (recordWriterInstantiated) {
// Hive ORC initializes its private `writer` field at the first write.
OrcFileFormat.addSparkVersionMetadata(recordWriter)
recordWriter.close(Reporter.NULL)
}
}
}
private[orc] object OrcFileFormat extends HiveInspectors with Logging {
// This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public.
private[orc] val SARG_PUSHDOWN = "sarg.pushdown"
// The extensions for ORC compression codecs
val extensionsForCompressionCodecNames = Map(
"NONE" -> "",
"SNAPPY" -> ".snappy",
"ZLIB" -> ".zlib",
"LZO" -> ".lzo")
def unwrapOrcStructs(
conf: Configuration,
dataSchema: StructType,
requiredSchema: StructType,
maybeStructOI: Option[StructObjectInspector],
iterator: Iterator[Writable]): Iterator[InternalRow] = {
val deserializer = new OrcSerde
val mutableRow = new SpecificInternalRow(requiredSchema.map(_.dataType))
val unsafeProjection = UnsafeProjection.create(requiredSchema)
def unwrap(oi: StructObjectInspector): Iterator[InternalRow] = {
val (fieldRefs, fieldOrdinals) = requiredSchema.zipWithIndex.map {
case (field, ordinal) =>
var ref = oi.getStructFieldRef(field.name)
if (ref == null) {
ref = oi.getStructFieldRef("_col" + dataSchema.fieldIndex(field.name))
}
ref -> ordinal
}.unzip
val unwrappers = fieldRefs.map(r => if (r == null) null else unwrapperFor(r))
iterator.map { value =>
val raw = deserializer.deserialize(value)
var i = 0
val length = fieldRefs.length
while (i < length) {
val fieldRef = fieldRefs(i)
val fieldValue = if (fieldRef == null) null else oi.getStructFieldData(raw, fieldRef)
if (fieldValue == null) {
mutableRow.setNullAt(fieldOrdinals(i))
} else {
unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i))
}
i += 1
}
unsafeProjection(mutableRow)
}
}
maybeStructOI.map(unwrap).getOrElse(Iterator.empty)
}
def setRequiredColumns(
conf: Configuration, dataSchema: StructType, requestedSchema: StructType): Unit = {
val ids = requestedSchema.map(a => dataSchema.fieldIndex(a.name): Integer)
val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip
HiveShim.appendReadColumns(conf, sortedIDs, sortedNames)
}
/**
* Add a metadata specifying Spark version.
*/
def addSparkVersionMetadata(recordWriter: RecordWriter[NullWritable, Writable]): Unit = {
try {
val writerField = recordWriter.getClass.getDeclaredField("writer")
writerField.setAccessible(true)
val writer = writerField.get(recordWriter).asInstanceOf[Writer]
writer.addUserMetadata(SPARK_VERSION_METADATA_KEY, UTF_8.encode(SPARK_VERSION_SHORT))
} catch {
case NonFatal(e) => log.warn(e.toString, e)
}
}
}