-
Notifications
You must be signed in to change notification settings - Fork 244
/
Copy pathGpuParquetScan.scala
2887 lines (2666 loc) · 122 KB
/
GpuParquetScan.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
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Copyright (c) 2019-2024, NVIDIA CORPORATION.
*
* Licensed 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 com.nvidia.spark.rapids
import java.io.{Closeable, EOFException, FileNotFoundException, IOException, OutputStream}
import java.net.URI
import java.nio.ByteBuffer
import java.nio.channels.SeekableByteChannel
import java.nio.charset.StandardCharsets
import java.util
import java.util.{Collections, Locale}
import java.util.concurrent._
import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.language.implicitConversions
import ai.rapids.cudf._
import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
import com.nvidia.spark.rapids.GpuMetric._
import com.nvidia.spark.rapids.ParquetPartitionReader.{CopyRange, LocalCopy}
import com.nvidia.spark.rapids.RapidsConf.ParquetFooterReaderType
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.filecache.FileCache
import com.nvidia.spark.rapids.jni.{DateTimeRebase, ParquetFooter}
import com.nvidia.spark.rapids.shims.{ColumnDefaultValuesShims, GpuParquetCrypto, GpuTypeShims, ParquetLegacyNanoAsLongShims, ParquetSchemaClipShims, ParquetStringPredShims, ReaderUtils, ShimFilePartitionReaderFactory, SparkShimImpl}
import org.apache.commons.io.IOUtils
import org.apache.commons.io.output.{CountingOutputStream, NullOutputStream}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataInputStream, Path}
import org.apache.parquet.bytes.BytesUtils
import org.apache.parquet.bytes.BytesUtils.readIntLittleEndian
import org.apache.parquet.column.ColumnDescriptor
import org.apache.parquet.filter2.predicate.FilterApi
import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
import org.apache.parquet.hadoop.ParquetFileWriter.MAGIC
import org.apache.parquet.hadoop.metadata._
import org.apache.parquet.io.{InputFile, SeekableInputStream}
import org.apache.parquet.schema.{DecimalMetadata, GroupType, MessageType, OriginalType, PrimitiveType, Type}
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
import org.apache.spark.TaskContext
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, PartitioningAwareFileIndex, SchemaColumnConvertNotSupportedException}
import org.apache.spark.sql.execution.datasources.v2.FileScan
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.execution.TrampolineUtil
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector => SparkVector}
import org.apache.spark.util.SerializableConfiguration
/**
* Base GpuParquetScan used for common code across Spark versions. Gpu version of
* Spark's 'ParquetScan'.
*
* @param sparkSession SparkSession.
* @param hadoopConf Hadoop configuration.
* @param fileIndex File index of the relation.
* @param dataSchema Schema of the data.
* @param readDataSchema Schema to read.
* @param readPartitionSchema Partition schema.
* @param pushedFilters Filters on non-partition columns.
* @param options Parquet option settings.
* @param partitionFilters Filters on partition columns.
* @param dataFilters File source metadata filters.
* @param rapidsConf Rapids configuration.
* @param queryUsesInputFile This is a parameter to easily allow turning it
* off in GpuTransitionOverrides if InputFileName,
* InputFileBlockStart, or InputFileBlockLength are used
*/
case class GpuParquetScan(
sparkSession: SparkSession,
hadoopConf: Configuration,
fileIndex: PartitioningAwareFileIndex,
dataSchema: StructType,
readDataSchema: StructType,
readPartitionSchema: StructType,
pushedFilters: Array[Filter],
options: CaseInsensitiveStringMap,
partitionFilters: Seq[Expression],
dataFilters: Seq[Expression],
rapidsConf: RapidsConf,
queryUsesInputFile: Boolean = false)
extends FileScan with GpuScan with Logging {
override def isSplitable(path: Path): Boolean = true
override def createReaderFactory(): PartitionReaderFactory = {
val broadcastedConf = sparkSession.sparkContext.broadcast(
new SerializableConfiguration(hadoopConf))
if (rapidsConf.isParquetPerFileReadEnabled) {
logInfo("Using the original per file parquet reader")
GpuParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf,
dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics,
options.asScala.toMap, None)
} else {
GpuParquetMultiFilePartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf,
dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics,
queryUsesInputFile, None)
}
}
override def equals(obj: Any): Boolean = obj match {
case p: GpuParquetScan =>
super.equals(p) && dataSchema == p.dataSchema && options == p.options &&
equivalentFilters(pushedFilters, p.pushedFilters) && rapidsConf == p.rapidsConf &&
queryUsesInputFile == p.queryUsesInputFile
case _ => false
}
override def hashCode(): Int = getClass.hashCode()
override def description(): String = {
super.description() + ", PushedFilters: " + seqToString(pushedFilters)
}
// overrides nothing in 330
def withFilters(
partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan =
this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters)
override def withInputFile(): GpuScan = copy(queryUsesInputFile = true)
}
object GpuParquetScan {
def tagSupport(scanMeta: ScanMeta[ParquetScan]): Unit = {
val scan = scanMeta.wrapped
val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema)
tagSupport(scan.sparkSession, schema, scanMeta)
}
def tagSupport(sparkSession: SparkSession, readSchema: StructType,
meta: RapidsMeta[_, _, _]): Unit = {
if (ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong) {
meta.willNotWorkOnGpu("GPU does not support spark.sql.legacy.parquet.nanosAsLong")
}
if (!meta.conf.isParquetEnabled) {
meta.willNotWorkOnGpu("Parquet input and output has been disabled. To enable set" +
s"${RapidsConf.ENABLE_PARQUET} to true")
}
if (!meta.conf.isParquetReadEnabled) {
meta.willNotWorkOnGpu("Parquet input has been disabled. To enable set" +
s"${RapidsConf.ENABLE_PARQUET_READ} to true")
}
FileFormatChecks.tag(meta, readSchema, ParquetFormatType, ReadFileOp)
// Currently timestamp conversion is not supported.
// If support needs to be added then we need to follow the logic in Spark's
// ParquetPartitionReaderFactory and VectorizedColumnReader which essentially
// does the following:
// - check if Parquet file was created by "parquet-mr"
// - if not then look at SQLConf.SESSION_LOCAL_TIMEZONE and assume timestamps
// were written in that timezone and convert them to UTC timestamps.
// Essentially this should boil down to a vector subtract of the scalar delta
// between the configured timezone's delta from UTC on the timestamp data.
val schemaHasTimestamps = readSchema.exists { field =>
TrampolineUtil.dataTypeExistsRecursively(field.dataType, _.isInstanceOf[TimestampType])
}
if (schemaHasTimestamps && sparkSession.sessionState.conf.isParquetINT96TimestampConversion) {
meta.willNotWorkOnGpu("GpuParquetScan does not support int96 timestamp conversion")
}
if (ColumnDefaultValuesShims.hasExistenceDefaultValues(readSchema)) {
meta.willNotWorkOnGpu("GpuParquetScan does not support default values in schema")
}
}
/**
* This estimates the number of nodes in a parquet footer schema based off of the parquet spec
* Specifically https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
*/
private def numNodesEstimate(dt: DataType): Long = dt match {
case StructType(fields) =>
// A struct has a group node that holds the children
1 + fields.map(f => numNodesEstimate(f.dataType)).sum
case ArrayType(elementType, _) =>
// A List/Array has one group node to tag it as a list and another one
// that is marked as repeating.
2 + numNodesEstimate(elementType)
case MapType(keyType, valueType, _) =>
// A Map has one group node to tag it as a map and another one
// that is marked as repeating, but holds the key/value
2 + numNodesEstimate(keyType) + numNodesEstimate(valueType)
case _ =>
// All the other types are just value types and are represented by a non-group node
1
}
/**
* Adjust the footer reader type based off of a heuristic.
*/
def footerReaderHeuristic(
inputValue: ParquetFooterReaderType.Value,
data: StructType,
read: StructType,
useFieldId: Boolean): ParquetFooterReaderType.Value = {
val canUseNative = !(useFieldId && ParquetSchemaClipShims.hasFieldIds(read))
if (!canUseNative) {
// Native does not support field IDs yet
ParquetFooterReaderType.JAVA
} else {
inputValue match {
case ParquetFooterReaderType.AUTO =>
val dnc = numNodesEstimate(data)
val rnc = numNodesEstimate(read)
if (rnc.toDouble / dnc <= 0.5 && dnc - rnc > 10) {
ParquetFooterReaderType.NATIVE
} else {
ParquetFooterReaderType.JAVA
}
case other => other
}
}
}
def throwIfRebaseNeededInExceptionMode(table: Table, dateRebaseMode: DateTimeRebaseMode,
timestampRebaseMode: DateTimeRebaseMode): Unit = {
(0 until table.getNumberOfColumns).foreach { i =>
val col = table.getColumn(i)
if (dateRebaseMode == DateTimeRebaseException &&
DateTimeRebaseUtils.isDateRebaseNeededInRead(col)) {
throw DataSourceUtils.newRebaseExceptionInRead("Parquet")
} else if (timestampRebaseMode == DateTimeRebaseException &&
DateTimeRebaseUtils.isTimeRebaseNeededInRead(col)) {
throw DataSourceUtils.newRebaseExceptionInRead("Parquet")
}
}
}
def rebaseDateTime(table: Table, dateRebaseMode: DateTimeRebaseMode,
timestampRebaseMode: DateTimeRebaseMode): Table = {
val dateRebaseNeeded = dateRebaseMode == DateTimeRebaseLegacy
val timeRebaseNeeded = timestampRebaseMode == DateTimeRebaseLegacy
lazy val tableHasDate = (0 until table.getNumberOfColumns).exists { i =>
checkTypeRecursively(table.getColumn(i), { dt => dt == DType.TIMESTAMP_DAYS })
}
lazy val tableHasTimestamp = (0 until table.getNumberOfColumns).exists { i =>
checkTypeRecursively(table.getColumn(i), { dt => dt == DType.TIMESTAMP_MICROSECONDS })
}
if ((dateRebaseNeeded && tableHasDate) || (timeRebaseNeeded && tableHasTimestamp)) {
// Need to close the input table when returning a new table.
withResource(table) { tmpTable =>
val newColumns = (0 until tmpTable.getNumberOfColumns).map { i =>
deepTransformRebaseDateTime(tmpTable.getColumn(i), dateRebaseNeeded, timeRebaseNeeded)
}
withResource(newColumns) { newCols =>
new Table(newCols: _*)
}
}
} else {
table
}
}
private def checkTypeRecursively(input: ColumnView, f: DType => Boolean): Boolean = {
val dt = input.getType
if (dt.isTimestampType && dt != DType.TIMESTAMP_DAYS && dt != DType.TIMESTAMP_MICROSECONDS) {
// There should be something wrong here since timestamps other than DAYS should already
// been converted into MICROSECONDS when reading Parquet files.
throw new IllegalStateException(s"Unexpected date/time type: $dt " +
"(expected TIMESTAMP_DAYS or TIMESTAMP_MICROSECONDS)")
}
dt match {
case DType.LIST | DType.STRUCT => (0 until input.getNumChildren).exists(i =>
withResource(input.getChildColumnView(i)) { child =>
checkTypeRecursively(child, f)
})
case t: DType => f(t)
}
}
private def deepTransformRebaseDateTime(cv: ColumnVector, dateRebaseNeeded: Boolean,
timeRebaseNeeded: Boolean): ColumnVector = {
ColumnCastUtil.deepTransform(cv) {
case (cv, _) if cv.getType.isTimestampType =>
// cv type is guaranteed to be either TIMESTAMP_DAYS or TIMESTAMP_MICROSECONDS,
// since we already checked it in `checkTypeRecursively`.
if ((cv.getType == DType.TIMESTAMP_DAYS && dateRebaseNeeded) ||
cv.getType == DType.TIMESTAMP_MICROSECONDS && timeRebaseNeeded) {
DateTimeRebase.rebaseJulianToGregorian(cv)
} else {
cv.copyToColumnVector()
}
}
}
}
// contains meta about all the blocks in a file
case class ParquetFileInfoWithBlockMeta(filePath: Path, blocks: collection.Seq[BlockMetaData],
partValues: InternalRow, schema: MessageType, readSchema: StructType,
dateRebaseMode: DateTimeRebaseMode, timestampRebaseMode: DateTimeRebaseMode,
hasInt96Timestamps: Boolean)
private case class BlockMetaWithPartFile(meta: ParquetFileInfoWithBlockMeta, file: PartitionedFile)
/**
* A parquet compatible stream that allows reading from a HostMemoryBuffer to Parquet.
* The majority of the code here was copied from Parquet's DelegatingSeekableInputStream with
* minor modifications to have it be make it Scala and call into the
* HostMemoryInputStreamMixIn's state.
*/
class HMBSeekableInputStream(
val hmb: HostMemoryBuffer,
val hmbLength: Long) extends SeekableInputStream
with HostMemoryInputStreamMixIn {
private val temp = new Array[Byte](8192)
override def seek(offset: Long): Unit = {
pos = offset
}
@throws[IOException]
override def readFully(buffer: Array[Byte]): Unit = {
val amountRead = read(buffer)
val remaining = buffer.length - amountRead
if (remaining > 0) {
throw new EOFException("Reached the end of stream with " + remaining + " bytes left to read")
}
}
@throws[IOException]
override def readFully(buffer: Array[Byte], offset: Int, length: Int): Unit = {
val amountRead = read(buffer, offset, length)
val remaining = length - amountRead
if (remaining > 0) {
throw new EOFException("Reached the end of stream with " + remaining + " bytes left to read")
}
}
@throws[IOException]
override def read(buf: ByteBuffer): Int =
if (buf.hasArray) {
readHeapBuffer(buf)
} else {
readDirectBuffer(buf)
}
@throws[IOException]
override def readFully(buf: ByteBuffer): Unit = {
if (buf.hasArray) {
readFullyHeapBuffer(buf)
} else {
readFullyDirectBuffer(buf)
}
}
private def readHeapBuffer(buf: ByteBuffer) = {
val bytesRead = read(buf.array, buf.arrayOffset + buf.position(), buf.remaining)
if (bytesRead < 0) {
bytesRead
} else {
buf.position(buf.position() + bytesRead)
bytesRead
}
}
private def readFullyHeapBuffer(buf: ByteBuffer): Unit = {
readFully(buf.array, buf.arrayOffset + buf.position(), buf.remaining)
buf.position(buf.limit)
}
private def readDirectBuffer(buf: ByteBuffer): Int = {
var nextReadLength = Math.min(buf.remaining, temp.length)
var totalBytesRead = 0
var bytesRead = 0
totalBytesRead = 0
bytesRead = read(temp, 0, nextReadLength)
while (bytesRead == temp.length) {
buf.put(temp)
totalBytesRead += bytesRead
nextReadLength = Math.min(buf.remaining, temp.length)
bytesRead = read(temp, 0, nextReadLength)
}
if (bytesRead < 0) {
if (totalBytesRead == 0) {
-1
} else {
totalBytesRead
}
} else {
buf.put(temp, 0, bytesRead)
totalBytesRead += bytesRead
totalBytesRead
}
}
private def readFullyDirectBuffer(buf: ByteBuffer): Unit = {
var nextReadLength = Math.min(buf.remaining, temp.length)
var bytesRead = 0
bytesRead = 0
bytesRead = read(temp, 0, nextReadLength)
while (nextReadLength > 0 && bytesRead >= 0) {
buf.put(temp, 0, bytesRead)
nextReadLength = Math.min(buf.remaining, temp.length)
bytesRead = read(temp, 0, nextReadLength)
}
if (bytesRead < 0 && buf.remaining > 0) {
throw new EOFException("Reached the end of stream with " +
buf.remaining + " bytes left to read")
}
}
}
class HMBInputFile(buffer: HostMemoryBuffer) extends InputFile {
override def getLength: Long = buffer.getLength
override def newStream(): SeekableInputStream = new HMBSeekableInputStream(buffer, getLength)
}
private case class GpuParquetFileFilterHandler(
@transient sqlConf: SQLConf,
metrics: Map[String, GpuMetric]) extends Logging {
private val FOOTER_LENGTH_SIZE = 4
private val isCaseSensitive = sqlConf.caseSensitiveAnalysis
private val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
private val pushDownDate = sqlConf.parquetFilterPushDownDate
private val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
private val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
// From Spark 340, more string predicates are supported as push-down filters, so this
// flag is renamed to 'xxxxStringPredicate' and specified by another config.
private val pushDownStringPredicate = ParquetStringPredShims.pushDown(sqlConf)
private val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
private val datetimeRebaseMode = SparkShimImpl.parquetRebaseRead(sqlConf)
private val int96RebaseMode = SparkShimImpl.int96ParquetRebaseRead(sqlConf)
private val readUseFieldId = ParquetSchemaClipShims.useFieldId(sqlConf)
private val ignoreMissingParquetFieldId = ParquetSchemaClipShims.ignoreMissingIds(sqlConf)
private val PARQUET_ENCRYPTION_CONFS = Seq("parquet.encryption.kms.client.class",
"parquet.encryption.kms.client.class", "parquet.crypto.factory.class")
private val PARQUET_MAGIC_ENCRYPTED = "PARE".getBytes(StandardCharsets.US_ASCII)
private def isParquetTimeInInt96(parquetType: Type): Boolean = {
parquetType match {
case p:PrimitiveType =>
p.getPrimitiveTypeName == PrimitiveTypeName.INT96
case g:GroupType => //GroupType
g.getFields.asScala.exists(t => isParquetTimeInInt96(t))
case _ => false
}
}
/**
* Convert the spark data type to something that the native processor can understand.
*/
private def convertToParquetNative(schema: DataType): ParquetFooter.SchemaElement = {
schema match {
case cst: StructType =>
val schemaBuilder = ParquetFooter.StructElement.builder()
cst.fields.foreach { field =>
schemaBuilder.addChild(field.name, convertToParquetNative(field.dataType))
}
schemaBuilder.build()
case _: NumericType | BinaryType | BooleanType | DateType | TimestampType | StringType =>
new ParquetFooter.ValueElement()
case at: ArrayType =>
new ParquetFooter.ListElement(convertToParquetNative(at.elementType))
case mt: MapType =>
new ParquetFooter.MapElement(
convertToParquetNative(mt.keyType),
convertToParquetNative(mt.valueType))
case other =>
throw new UnsupportedOperationException(s"Need some help here $other...")
}
}
private def convertToFooterSchema(schema: StructType): ParquetFooter.StructElement = {
convertToParquetNative(schema).asInstanceOf[ParquetFooter.StructElement]
}
private def getFooterBuffer(
filePath: Path,
conf: Configuration,
metrics: Map[String, GpuMetric]): HostMemoryBuffer = {
val filePathString = filePath.toString
FileCache.get.getFooter(filePathString, conf).map { hmb =>
withResource(hmb) { _ =>
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_HITS, NoopMetric) += 1
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_HITS_SIZE, NoopMetric) += hmb.getLength
// buffer includes header and trailing length and magic, stripped here
hmb.slice(MAGIC.length, hmb.getLength - Integer.BYTES - MAGIC.length)
}
}.getOrElse {
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_MISSES, NoopMetric) += 1
withResource(readFooterBuffer(filePath, conf)) { hmb =>
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_MISSES_SIZE, NoopMetric) += hmb.getLength
// footer was not cached, so try to cache it
// If we get a filecache token then we can complete the caching by providing the data.
// If we do not get a token then we should not cache this data.
val cacheToken = FileCache.get.startFooterCache(filePathString, conf)
cacheToken.foreach { t =>
t.complete(hmb.slice(0, hmb.getLength))
}
// buffer includes header and trailing length and magic, stripped here
hmb.slice(MAGIC.length, hmb.getLength - Integer.BYTES - MAGIC.length)
}
}
}
private def readFooterBuffer(
filePath: Path,
conf: Configuration): HostMemoryBuffer = {
PerfIO.readParquetFooterBuffer(filePath, conf, verifyParquetMagic)
.getOrElse(readFooterBufUsingHadoop(filePath, conf))
}
private def readFooterBufUsingHadoop(filePath: Path, conf: Configuration): HostMemoryBuffer = {
val fs = filePath.getFileSystem(conf)
val stat = fs.getFileStatus(filePath)
// Much of this code came from the parquet_mr projects ParquetFileReader, and was modified
// to match our needs
val fileLen = stat.getLen
// MAGIC + data + footer + footerIndex + MAGIC
if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) {
throw new RuntimeException(s"$filePath is not a Parquet file (too small length: $fileLen )")
}
val footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length
withResource(fs.open(filePath)) { inputStream =>
withResource(new NvtxRange("ReadFooterBytes", NvtxColor.YELLOW)) { _ =>
inputStream.seek(footerLengthIndex)
val footerLength = readIntLittleEndian(inputStream)
val magic = new Array[Byte](MAGIC.length)
inputStream.readFully(magic)
val footerIndex = footerLengthIndex - footerLength
verifyParquetMagic(filePath, magic)
if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) {
throw new RuntimeException(s"corrupted file: the footer index is not within " +
s"the file: $footerIndex")
}
val hmbLength = (fileLen - footerIndex).toInt
closeOnExcept(HostMemoryBuffer.allocate(hmbLength + MAGIC.length, false)) { outBuffer =>
val out = new HostMemoryOutputStream(outBuffer)
out.write(MAGIC)
inputStream.seek(footerIndex)
// read the footer til the end of the file
val tmpBuffer = new Array[Byte](4096)
var bytesLeft = hmbLength
while (bytesLeft > 0) {
val readLength = Math.min(bytesLeft, tmpBuffer.length)
inputStream.readFully(tmpBuffer, 0, readLength)
out.write(tmpBuffer, 0, readLength)
bytesLeft -= readLength
}
outBuffer
}
}
}
}
private def verifyParquetMagic(filePath: Path, magic: Array[Byte]): Unit = {
if (!util.Arrays.equals(MAGIC, magic)) {
if (util.Arrays.equals(PARQUET_MAGIC_ENCRYPTED, magic)) {
throw new RuntimeException("The GPU does not support reading encrypted Parquet " +
"files. To read encrypted or columnar encrypted files, disable the GPU Parquet " +
s"reader via ${RapidsConf.ENABLE_PARQUET_READ.key}.")
} else {
throw new RuntimeException(s"$filePath is not a Parquet file. " +
s"Expected magic number at tail ${util.Arrays.toString(MAGIC)} " +
s"but found ${util.Arrays.toString(magic)}")
}
}
}
private def readAndFilterFooter(
file: PartitionedFile,
conf : Configuration,
readDataSchema: StructType,
filePath: Path): ParquetFooter = {
val footerSchema = convertToFooterSchema(readDataSchema)
val footerBuffer = getFooterBuffer(filePath, conf, metrics)
withResource(footerBuffer) { footerBuffer =>
withResource(new NvtxRange("Parse and filter footer by range", NvtxColor.RED)) { _ =>
// In the future, if we know we're going to read the entire file,
// i.e.: having file length from file stat == amount to read,
// then sending -1 as length to the native footer parser allows it to
// skip the row group filtering.
val len = file.length
ParquetFooter.readAndFilter(footerBuffer, file.start, len,
footerSchema, !isCaseSensitive)
}
}
}
@scala.annotation.nowarn(
"msg=method readFooter in class ParquetFileReader is deprecated"
)
private def readAndSimpleFilterFooter(
file: PartitionedFile,
conf : Configuration,
filePath: Path): ParquetMetadata = {
//noinspection ScalaDeprecation
withResource(new NvtxRange("readFooter", NvtxColor.YELLOW)) { _ =>
val filePathString = filePath.toString
FileCache.get.getFooter(filePathString, conf).map { hmb =>
withResource(hmb) { _ =>
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_HITS, NoopMetric) += 1
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_HITS_SIZE, NoopMetric) += hmb.getLength
ParquetFileReader.readFooter(new HMBInputFile(hmb),
ParquetMetadataConverter.range(file.start, file.start + file.length))
}
}.getOrElse {
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_MISSES, NoopMetric) += 1
// footer was not cached, so try to cache it
// If we get a filecache token then we can complete the caching by providing the data.
// If something goes wrong before completing the caching then the token must be canceled.
// If we do not get a token then we should not cache this data.
val cacheToken = FileCache.get.startFooterCache(filePathString, conf)
cacheToken.map { token =>
var needTokenCancel = true
try {
withResource(readFooterBuffer(filePath, conf)) { hmb =>
metrics.getOrElse(GpuMetric.FILECACHE_FOOTER_MISSES_SIZE, NoopMetric) += hmb.getLength
token.complete(hmb.slice(0, hmb.getLength))
needTokenCancel = false
ParquetFileReader.readFooter(new HMBInputFile(hmb),
ParquetMetadataConverter.range(file.start, file.start + file.length))
}
} finally {
if (needTokenCancel) {
token.cancel()
}
}
}.getOrElse {
ParquetFileReader.readFooter(conf, filePath,
ParquetMetadataConverter.range(file.start, file.start + file.length))
}
}
}
}
@scala.annotation.nowarn
def filterBlocks(
footerReader: ParquetFooterReaderType.Value,
file: PartitionedFile,
conf: Configuration,
filters: Array[Filter],
readDataSchema: StructType): ParquetFileInfoWithBlockMeta = {
withResource(new NvtxRange("filterBlocks", NvtxColor.PURPLE)) { _ =>
val filePath = new Path(new URI(file.filePath.toString()))
// Make sure we aren't trying to read encrypted files. For now, remove the related
// parquet confs from the hadoop configuration and try to catch the resulting
// exception and print a useful message
PARQUET_ENCRYPTION_CONFS.foreach { encryptConf =>
if (conf.get(encryptConf) != null) {
conf.unset(encryptConf)
}
}
val fileHadoopConf =
ReaderUtils.getHadoopConfForReaderThread(new Path(file.filePath.toString), conf)
val footer: ParquetMetadata = try {
footerReader match {
case ParquetFooterReaderType.NATIVE =>
val serialized = withResource(readAndFilterFooter(file, fileHadoopConf,
readDataSchema, filePath)) { tableFooter =>
if (tableFooter.getNumColumns <= 0) {
// Special case because java parquet reader does not like having 0 columns.
val numRows = tableFooter.getNumRows
val block = new BlockMetaData()
block.setRowCount(numRows)
val schema = new MessageType("root")
return ParquetFileInfoWithBlockMeta(filePath, Seq(block), file.partitionValues,
schema, readDataSchema, DateTimeRebaseLegacy, DateTimeRebaseLegacy,
hasInt96Timestamps = false)
}
tableFooter.serializeThriftFile()
}
withResource(serialized) { serialized =>
withResource(new NvtxRange("readFilteredFooter", NvtxColor.YELLOW)) { _ =>
val inputFile = new HMBInputFile(serialized)
// We already filtered the ranges so no need to do more here...
ParquetFileReader.readFooter(inputFile, ParquetMetadataConverter.NO_FILTER)
}
}
case _ =>
readAndSimpleFilterFooter(file, fileHadoopConf, filePath)
}
} catch {
case e if GpuParquetCrypto.isColumnarCryptoException(e) =>
throw new RuntimeException("The GPU does not support reading encrypted Parquet " +
"files. To read encrypted or columnar encrypted files, disable the GPU Parquet " +
s"reader via ${RapidsConf.ENABLE_PARQUET_READ.key}.", e)
}
val fileSchema = footer.getFileMetaData.getSchema
// check spark.sql.parquet.fieldId.read.ignoreMissing
ParquetSchemaClipShims.checkIgnoreMissingIds(ignoreMissingParquetFieldId, fileSchema,
readDataSchema)
val pushedFilters = if (enableParquetFilterPushDown) {
val parquetFilters = SparkShimImpl.getParquetFilters(fileSchema, pushDownDate,
pushDownTimestamp, pushDownDecimal, pushDownStringPredicate, pushDownInFilterThreshold,
isCaseSensitive, footer.getFileMetaData.getKeyValueMetaData.get, datetimeRebaseMode)
filters.flatMap(parquetFilters.createFilter).reduceOption(FilterApi.and)
} else {
None
}
val blocks = if (pushedFilters.isDefined) {
withResource(new NvtxRange("getBlocksWithFilter", NvtxColor.CYAN)) { _ =>
// Use the ParquetFileReader to perform dictionary-level filtering
ParquetInputFormat.setFilterPredicate(fileHadoopConf, pushedFilters.get)
//noinspection ScalaDeprecation
withResource(new ParquetFileReader(fileHadoopConf, footer.getFileMetaData, filePath,
footer.getBlocks, Collections.emptyList[ColumnDescriptor])) { parquetReader =>
parquetReader.getRowGroups
}
}
} else {
footer.getBlocks
}
val (clipped, clippedSchema) =
withResource(new NvtxRange("clipSchema", NvtxColor.DARK_GREEN)) { _ =>
val clippedSchema = ParquetSchemaUtils.clipParquetSchema(
fileSchema, readDataSchema, isCaseSensitive, readUseFieldId)
// Check if the read schema is compatible with the file schema.
checkSchemaCompat(clippedSchema, readDataSchema,
(t: Type, d: DataType) => throwTypeIncompatibleError(t, d, file.filePath.toString()),
isCaseSensitive, readUseFieldId)
val clipped = GpuParquetUtils.clipBlocksToSchema(clippedSchema, blocks, isCaseSensitive)
(clipped, clippedSchema)
}
val hasDateTimeInReadSchema = DataTypeUtils.hasDateOrTimestampType(readDataSchema)
val dateRebaseModeForThisFile = DateTimeRebaseUtils.datetimeRebaseMode(
footer.getFileMetaData.getKeyValueMetaData.get,
datetimeRebaseMode,
hasDateTimeInReadSchema)
val hasInt96Timestamps = isParquetTimeInInt96(fileSchema)
val timestampRebaseModeForThisFile = if (hasInt96Timestamps) {
DateTimeRebaseUtils.int96RebaseMode(
footer.getFileMetaData.getKeyValueMetaData.get, int96RebaseMode)
} else {
dateRebaseModeForThisFile
}
ParquetFileInfoWithBlockMeta(filePath, clipped, file.partitionValues,
clippedSchema, readDataSchema, dateRebaseModeForThisFile,
timestampRebaseModeForThisFile, hasInt96Timestamps)
}
}
/**
* Recursively check if the read schema is compatible with the file schema. The errorCallback
* will be invoked to throw an exception once any incompatible type pairs are found.
*
* Any element in the read schema that are missing from the file schema are ignored.
*
* The function only accepts top-level schemas, which means structures of root columns. Based
* on this assumption, it can infer root types from input schemas.
*
* @param fileType input file's Parquet schema
* @param readType spark type read from Parquet file
* @param errorCallback call back function to throw exception if type mismatch
* @param rootFileType file type of each root column
* @param rootReadType read type of each root column
*/
private def checkSchemaCompat(fileType: Type,
readType: DataType,
errorCallback: (Type, DataType) => Unit,
isCaseSensitive: Boolean,
useFieldId: Boolean,
rootFileType: Option[Type] = None,
rootReadType: Option[DataType] = None): Unit = {
readType match {
case struct: StructType =>
val fileFieldMap = fileType.asGroupType().getFields.asScala
.map { f =>
(if (isCaseSensitive) f.getName else f.getName.toLowerCase(Locale.ROOT)) -> f
}.toMap
val fieldIdToFieldMap = ParquetSchemaClipShims.fieldIdToFieldMap(useFieldId, fileType)
def getParquetType(f: StructField): Option[Type] = {
if(useFieldId && ParquetSchemaClipShims.hasFieldId(f)) {
// use field ID and Spark schema specified field ID
fieldIdToFieldMap.get(ParquetSchemaClipShims.getFieldId(f))
} else {
fileFieldMap.get(if (isCaseSensitive) f.name else f.name.toLowerCase(Locale.ROOT))
}
}
struct.fields.foreach { f =>
getParquetType(f).foreach { fieldType =>
checkSchemaCompat(fieldType,
f.dataType,
errorCallback,
isCaseSensitive,
useFieldId,
// Record root types for each column, so as to throw a readable exception
// over nested types.
Some(rootFileType.getOrElse(fieldType)),
Some(rootReadType.getOrElse(f.dataType)))
}
}
case array: ArrayType =>
if (fileType.isPrimitive) {
if (fileType.getRepetition == Type.Repetition.REPEATED) {
checkSchemaCompat(fileType, array.elementType, errorCallback, isCaseSensitive,
useFieldId, rootFileType, rootReadType)
} else {
errorCallback(fileType, readType)
}
} else {
val fileGroupType = fileType.asGroupType()
if (fileGroupType.getFieldCount > 1 &&
fileGroupType.isRepetition(Type.Repetition.REPEATED)) {
// legacy array format where struct child is directly repeated under array type group
checkSchemaCompat(fileGroupType, array.elementType, errorCallback, isCaseSensitive,
useFieldId, rootFileType, rootReadType)
} else {
val repeatedType = fileGroupType.getType(0)
val childType =
if (isElementType(repeatedType, fileType.getName)) {
// Legacy element, per Parquet LogicalType backward compatibility rules.
// Retain the child as the element type.
repeatedType
}
else {
// Conforms to current Parquet LogicalType rules.
// Unwrap child group layer, and use grandchild's element type.
repeatedType.asGroupType().getType(0)
}
checkSchemaCompat(childType, array.elementType, errorCallback, isCaseSensitive,
useFieldId, rootFileType, rootReadType)
}
}
case map: MapType =>
val parquetMap = fileType.asGroupType().getType(0).asGroupType()
val parquetMapKey = parquetMap.getType(0)
val parquetMapValue = parquetMap.getType(1)
checkSchemaCompat(parquetMapKey, map.keyType, errorCallback, isCaseSensitive, useFieldId,
rootFileType, rootReadType)
checkSchemaCompat(parquetMapValue, map.valueType, errorCallback, isCaseSensitive,
useFieldId, rootFileType, rootReadType)
case dt =>
checkPrimitiveCompat(fileType.asPrimitiveType(),
dt,
() => errorCallback(rootFileType.get, rootReadType.get))
}
}
// scalastyle:off
// Implement Parquet LIST backwards-compatibility rules.
// See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules
// Inspired by Apache Spark's ParquetSchemaConverter.isElementType():
// https://github.com/apache/spark/blob/3a0e6bde2aaa11e1165f4fde040ff02e1743795e/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala#L413
// scalastyle:on
private def isElementType(repeatedType: Type, parentName: String): Boolean = {
{
// For legacy 2-level list types with primitive element type, e.g.:
//
// // ARRAY<INT> (nullable list, non-null elements)
// optional group my_list (LIST) {
// repeated int32 element;
// }
//
repeatedType.isPrimitive
} || {
// For legacy 2-level list types whose element type is a group type with 2 or more fields,
// e.g.:
//
// // ARRAY<STRUCT<str: STRING, num: INT>> (nullable list, non-null elements)
// optional group my_list (LIST) {
// repeated group element {
// required binary str (UTF8);
// required int32 num;
// };
// }
//
repeatedType.asGroupType().getFieldCount > 1
} || {
// For legacy 2-level list types generated by parquet-avro (Parquet version < 1.6.0), e.g.:
//
// // ARRAY<STRUCT<str: STRING>> (nullable list, non-null elements)
// optional group my_list (LIST) {
// repeated group array {
// required binary str (UTF8);
// };
// }
//
repeatedType.getName == "array"
} || {
// For Parquet data generated by parquet-thrift, e.g.:
//
// // ARRAY<STRUCT<str: STRING>> (nullable list, non-null elements)
// optional group my_list (LIST) {
// repeated group my_list_tuple {
// required binary str (UTF8);
// };
// }
//
repeatedType.getName == s"${parentName}_tuple"
}
}
/**
* Check the compatibility over primitive types. This function refers to the `getUpdater` method
* of org.apache.spark.sql.execution.datasources.parquet.ParquetVectorUpdaterFactory.
*
* To avoid unnecessary pattern matching, this function is designed to return or throw ASAP.
*
* This function uses some deprecated Parquet APIs, because Spark 3.1 is relied on parquet-mr
* of an older version.
*/
@scala.annotation.nowarn("msg=method getDecimalMetadata in class PrimitiveType is deprecated")
private def checkPrimitiveCompat(pt: PrimitiveType,
dt: DataType,
errorCallback: () => Unit): Unit = {
pt.getPrimitiveTypeName match {
case PrimitiveTypeName.BOOLEAN if dt == DataTypes.BooleanType =>
return
case PrimitiveTypeName.INT32 =>
if (dt == DataTypes.IntegerType || GpuTypeShims.isSupportedYearMonthType(dt)
|| canReadAsIntDecimal(pt, dt)) {
// Year-month interval type is stored as int32 in parquet
return
}
// TODO: After we deprecate Spark 3.1, replace OriginalType with LogicalTypeAnnotation
if (dt == DataTypes.LongType && pt.getOriginalType == OriginalType.UINT_32) {
return
}
if (dt == DataTypes.ByteType || dt == DataTypes.ShortType || dt == DataTypes.DateType) {
return
}
case PrimitiveTypeName.INT64 =>
if (dt == DataTypes.LongType || GpuTypeShims.isSupportedDayTimeType(dt) ||
// Day-time interval type is stored as int64 in parquet
canReadAsLongDecimal(pt, dt)) {
return
}
// TODO: After we deprecate Spark 3.1, replace OriginalType with LogicalTypeAnnotation
if (isLongDecimal(dt) && pt.getOriginalType == OriginalType.UINT_64) {
return
}
if (pt.getOriginalType == OriginalType.TIMESTAMP_MICROS ||
pt.getOriginalType == OriginalType.TIMESTAMP_MILLIS) {
return
}
case PrimitiveTypeName.FLOAT if dt == DataTypes.FloatType =>
return
case PrimitiveTypeName.DOUBLE if dt == DataTypes.DoubleType =>
return
case PrimitiveTypeName.INT96 if dt == DataTypes.TimestampType =>
return
case PrimitiveTypeName.BINARY if dt == DataTypes.StringType =>
// PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY for StringType is not supported by parquet
return
case PrimitiveTypeName.BINARY | PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
if dt == DataTypes.BinaryType =>
return