-
Notifications
You must be signed in to change notification settings - Fork 242
/
Copy pathGpuParquetScan.scala
2238 lines (2037 loc) · 92.7 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-2022, 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.{EOFException, FileNotFoundException, IOException, OutputStream}
import java.net.URI
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
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 scala.math.max
import ai.rapids.cudf._
import com.nvidia.spark.RebaseHelper
import com.nvidia.spark.rapids.GpuMetric._
import com.nvidia.spark.rapids.ParquetPartitionReader.CopyRange
import com.nvidia.spark.rapids.RapidsConf.ParquetFooterReaderType
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.jni.ParquetFooter
import com.nvidia.spark.rapids.shims.{GpuParquetCrypto, GpuTypeShims, ParquetFieldIdShims, SparkShimImpl}
import java.util
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, Types}
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.parquet.ParquetReadSupport
import org.apache.spark.sql.execution.datasources.v2.{FilePartitionReaderFactory, 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 ScanWithMetrics with FileScan 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)
} else {
GpuParquetMultiFilePartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf,
dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics,
queryUsesInputFile)
}
}
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)
}
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 throwIfNeeded(
table: Table,
isCorrectedInt96Rebase: Boolean,
isCorrectedDateTimeRebase: Boolean,
hasInt96Timestamps: Boolean): Unit = {
(0 until table.getNumberOfColumns).foreach { i =>
val col = table.getColumn(i)
// if col is a day
if (!isCorrectedDateTimeRebase && RebaseHelper.isDateRebaseNeededInRead(col)) {
throw DataSourceUtils.newRebaseExceptionInRead("Parquet")
}
// if col is a time
else if (hasInt96Timestamps && !isCorrectedInt96Rebase ||
!hasInt96Timestamps && !isCorrectedDateTimeRebase) {
if (RebaseHelper.isTimeRebaseNeededInRead(col)) {
throw DataSourceUtils.newRebaseExceptionInRead("Parquet")
}
}
}
}
def tagSupport(
sparkSession: SparkSession,
readSchema: StructType,
meta: RapidsMeta[_, _, _]): Unit = {
val sqlConf = sparkSession.conf
ParquetFieldIdShims.tagGpuSupportReadForFieldId(meta, sparkSession.sessionState.conf)
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)
val schemaHasStrings = readSchema.exists { field =>
TrampolineUtil.dataTypeExistsRecursively(field.dataType, _.isInstanceOf[StringType])
}
if (sqlConf.get(SQLConf.PARQUET_BINARY_AS_STRING.key,
SQLConf.PARQUET_BINARY_AS_STRING.defaultValueString).toBoolean && schemaHasStrings) {
meta.willNotWorkOnGpu(s"GpuParquetScan does not support" +
s" ${SQLConf.PARQUET_BINARY_AS_STRING.key}")
}
val schemaHasTimestamps = readSchema.exists { field =>
TrampolineUtil.dataTypeExistsRecursively(field.dataType, _.isInstanceOf[TimestampType])
}
def isTsOrDate(dt: DataType) : Boolean = dt match {
case TimestampType | DateType => true
case _ => false
}
val schemaMightNeedNestedRebase = readSchema.exists { field =>
if (DataTypeUtils.isNestedType(field.dataType)) {
TrampolineUtil.dataTypeExistsRecursively(field.dataType, isTsOrDate)
} else {
false
}
}
// 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.
if (schemaHasTimestamps && sparkSession.sessionState.conf.isParquetINT96TimestampConversion) {
meta.willNotWorkOnGpu("GpuParquetScan does not support int96 timestamp conversion")
}
sqlConf.get(SparkShimImpl.int96ParquetRebaseReadKey) match {
case "EXCEPTION" => if (schemaMightNeedNestedRebase) {
meta.willNotWorkOnGpu("Nested timestamp and date values are not supported when " +
s"${SparkShimImpl.int96ParquetRebaseReadKey} is EXCEPTION")
}
case "CORRECTED" => // Good
case "LEGACY" => // really is EXCEPTION for us...
if (schemaMightNeedNestedRebase) {
meta.willNotWorkOnGpu("Nested timestamp and date values are not supported when " +
s"${SparkShimImpl.int96ParquetRebaseReadKey} is LEGACY")
}
case other =>
meta.willNotWorkOnGpu(s"$other is not a supported read rebase mode")
}
sqlConf.get(SparkShimImpl.parquetRebaseReadKey) match {
case "EXCEPTION" => if (schemaMightNeedNestedRebase) {
meta.willNotWorkOnGpu("Nested timestamp and date values are not supported when " +
s"${SparkShimImpl.parquetRebaseReadKey} is EXCEPTION")
}
case "CORRECTED" => // Good
case "LEGACY" => // really is EXCEPTION for us...
if (schemaMightNeedNestedRebase) {
meta.willNotWorkOnGpu("Nested timestamp and date values are not supported when " +
s"${SparkShimImpl.parquetRebaseReadKey} is LEGACY")
}
case other =>
meta.willNotWorkOnGpu(s"$other is not a supported read rebase mode")
}
}
}
/**
* Base object that has common functions for both GpuParquetPartitionReaderFactory
* and GpuParquetPartitionReaderFactory
*/
object GpuParquetPartitionReaderFactoryBase {
def filterClippedSchema(
clippedSchema: MessageType,
fileSchema: MessageType,
isCaseSensitive: Boolean): MessageType = {
val fs = fileSchema.asGroupType()
val types = if (isCaseSensitive) {
val inFile = fs.getFields.asScala.map(_.getName).toSet
clippedSchema.asGroupType()
.getFields.asScala.filter(f => inFile.contains(f.getName))
} else {
val inFile = fs.getFields.asScala
.map(_.getName.toLowerCase(Locale.ROOT)).toSet
clippedSchema.asGroupType()
.getFields.asScala
.filter(f => inFile.contains(f.getName.toLowerCase(Locale.ROOT)))
}
if (types.isEmpty) {
Types.buildMessage().named("spark_schema")
} else {
Types
.buildMessage()
.addFields(types: _*)
.named("spark_schema")
}
}
// Copied from Spark
private val SPARK_VERSION_METADATA_KEY = "org.apache.spark.version"
// Copied from Spark
private val SPARK_LEGACY_DATETIME = "org.apache.spark.legacyDateTime"
// Copied from Spark
private val SPARK_LEGACY_INT96 = "org.apache.spark.legacyINT96"
def isCorrectedInt96RebaseMode(
lookupFileMeta: String => String,
isCorrectedInt96ModeConfig: Boolean): Boolean = {
// If there is no version, we return the mode specified by the config.
Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
// Files written by Spark 3.0 and earlier follow the legacy hybrid calendar and we need to
// rebase the INT96 timestamp values.
// Files written by Spark 3.1 and latter may also need the rebase if they were written with
// the "LEGACY" rebase mode.
if (version >= "3.1.0") {
lookupFileMeta(SPARK_LEGACY_INT96) == null
} else if (version >= "3.0.0") {
lookupFileMeta(SPARK_LEGACY_DATETIME) == null
} else {
false
}
}.getOrElse(isCorrectedInt96ModeConfig)
}
def isCorrectedRebaseMode(
lookupFileMeta: String => String,
isCorrectedModeConfig: Boolean): Boolean = {
// If there is no version, we return the mode specified by the config.
Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
// Files written by Spark 2.4 and earlier follow the legacy hybrid calendar and we need to
// rebase the datetime values.
// Files written by Spark 3.0 and later may also need the rebase if they were written with
// the "LEGACY" rebase mode.
version >= "3.0.0" && lookupFileMeta(SPARK_LEGACY_DATETIME) == null
}.getOrElse(isCorrectedModeConfig)
}
}
// contains meta about all the blocks in a file
private case class ParquetFileInfoWithBlockMeta(filePath: Path, blocks: Seq[BlockMetaData],
partValues: InternalRow, schema: MessageType, isCorrectedInt96RebaseMode: Boolean,
isCorrectedRebaseMode: Boolean, hasInt96Timestamps: Boolean)
/**
* 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) extends Arm {
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
private val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
private val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
private val rebaseMode = SparkShimImpl.parquetRebaseRead(sqlConf)
private val isCorrectedRebase = "CORRECTED" == rebaseMode
val int96RebaseMode = SparkShimImpl.int96ParquetRebaseRead(sqlConf)
private val isInt96CorrectedRebase = "CORRECTED" == int96RebaseMode
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)
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
}
}
private def addNamesAndCount(names: ArrayBuffer[String], children: ArrayBuffer[Int],
name: String, numChildren: Int): Unit = {
names += name
children += numChildren
}
/**
* Flatten a Spark schema according to the parquet standard. This does not work for older
* parquet files that did not fully follow the standard, or were before some of these
* things were standardized. This will be fixed as a part of
* https://github.com/NVIDIA/spark-rapids-jni/issues/210
*/
private def depthFirstNamesHelper(schema: DataType, elementName: String, makeLowerCase: Boolean,
names: ArrayBuffer[String], children: ArrayBuffer[Int]): Unit = {
val name = if (makeLowerCase) {
elementName.toLowerCase(Locale.ROOT)
} else {
elementName
}
schema match {
case cst: StructType =>
addNamesAndCount(names, children, name, cst.length)
cst.fields.foreach { field =>
depthFirstNamesHelper(field.dataType, field.name, makeLowerCase, names, children)
}
case _: NumericType | BinaryType | BooleanType | DateType | TimestampType | StringType =>
addNamesAndCount(names, children, name, 0)
case at: ArrayType =>
addNamesAndCount(names, children, name, 1)
addNamesAndCount(names, children, "list", 1)
depthFirstNamesHelper(at.elementType, "element", makeLowerCase, names, children)
case mt: MapType =>
addNamesAndCount(names, children, name, 1)
addNamesAndCount(names, children, "key_value", 2)
depthFirstNamesHelper(mt.keyType, "key", makeLowerCase, names, children)
depthFirstNamesHelper(mt.valueType, "value", makeLowerCase, names, children)
case other =>
throw new UnsupportedOperationException(s"Need some help here $other...")
}
}
def depthFirstNames(schema: StructType, makeLowerCase: Boolean): (Array[String], Array[Int]) = {
withResource(new NvtxRange("prepare schema", NvtxColor.WHITE)) { _ =>
// Initialize them with a quick length for non-nested values
val names = new ArrayBuffer[String](schema.length)
val children = new ArrayBuffer[Int](schema.length)
schema.fields.foreach { field =>
depthFirstNamesHelper(field.dataType, field.name, makeLowerCase, names, children)
}
(names.toArray, children.toArray)
}
}
def readAndFilterFooter(
file: PartitionedFile,
conf : Configuration,
readDataSchema: StructType,
filePath: Path): ParquetFooter = {
val (names, children) = depthFirstNames(readDataSchema, !isCaseSensitive)
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
val FOOTER_LENGTH_SIZE = 4
// 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
val footerBuffer = 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)
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)}")
}
}
val footerIndex = footerLengthIndex - footerLength
if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) {
throw new RuntimeException(s"corrupted file: the footer index is not within " +
s"the file: $footerIndex")
}
inputStream.seek(footerIndex)
closeOnExcept(HostMemoryBuffer.allocate(footerLength, false)) { outBuffer =>
val out = new HostMemoryOutputStream(outBuffer)
val tmpBuffer = new Array[Byte](4096)
var bytesLeft = footerLength
while (bytesLeft > 0) {
val readLength = Math.min(bytesLeft, tmpBuffer.length)
inputStream.readFully(tmpBuffer, 0, readLength)
out.write(tmpBuffer, 0, readLength)
bytesLeft -= readLength
}
outBuffer
}
}
}
withResource(footerBuffer) { footerBuffer =>
withResource(new NvtxRange("Parse and filter footer by range", NvtxColor.RED)) { _ =>
val len = if (fileLen <= file.length) {
// secret signal to skip filtering
-1
} else {
file.length
}
ParquetFooter.readAndFilter(footerBuffer, file.start, len,
names, children, readDataSchema.length, !isCaseSensitive)
}
}
}
def readAndSimpleFilterFooter(
file: PartitionedFile,
conf : Configuration,
filePath: Path): ParquetMetadata = {
//noinspection ScalaDeprecation
withResource(new NvtxRange("readFooter", NvtxColor.YELLOW)) { _ =>
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))
// 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 footer = try {
footerReader match {
case ParquetFooterReaderType.NATIVE =>
val serialized = withResource(readAndFilterFooter(file, conf,
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, false, false, 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, conf, 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
val pushedFilters = if (enableParquetFilterPushDown) {
val parquetFilters = SparkShimImpl.getParquetFilters(fileSchema, pushDownDate,
pushDownTimestamp, pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold,
isCaseSensitive, footer.getFileMetaData.getKeyValueMetaData.get, rebaseMode)
filters.flatMap(parquetFilters.createFilter).reduceOption(FilterApi.and)
} else {
None
}
val hasInt96Timestamps = isParquetTimeInInt96(fileSchema)
val isCorrectedRebaseForThisFile =
GpuParquetPartitionReaderFactoryBase.isCorrectedRebaseMode(
footer.getFileMetaData.getKeyValueMetaData.get, isCorrectedRebase)
val isCorrectedInt96RebaseForThisFile =
GpuParquetPartitionReaderFactoryBase.isCorrectedInt96RebaseMode(
footer.getFileMetaData.getKeyValueMetaData.get, isInt96CorrectedRebase)
val blocks = if (pushedFilters.isDefined) {
withResource(new NvtxRange("getBlocksWithFilter", NvtxColor.CYAN)) { _ =>
// Use the ParquetFileReader to perform dictionary-level filtering
ParquetInputFormat.setFilterPredicate(conf, pushedFilters.get)
//noinspection ScalaDeprecation
withResource(new ParquetFileReader(conf, 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 clippedSchemaTmp = ParquetReadSupport.clipParquetSchema(fileSchema, readDataSchema,
isCaseSensitive)
// Check if the read schema is compatible with the file schema.
checkSchemaCompat(clippedSchemaTmp, readDataSchema,
(t: Type, d: DataType) => throwTypeIncompatibleError(t, d, file.filePath),
isCaseSensitive)
// ParquetReadSupport.clipParquetSchema does most of what we want, but it includes
// everything in readDataSchema, even if it is not in fileSchema we want to remove those
// for our own purposes
val clippedSchema =
GpuParquetPartitionReaderFactoryBase.filterClippedSchema(clippedSchemaTmp,
fileSchema, isCaseSensitive)
val columnPaths = clippedSchema.getPaths.asScala.map(x => ColumnPath.get(x: _*))
val clipped =
ParquetPartitionReader.clipBlocks(columnPaths, blocks.asScala, isCaseSensitive)
(clipped, clippedSchema)
}
ParquetFileInfoWithBlockMeta(filePath, clipped, file.partitionValues,
clippedSchema, isCorrectedInt96RebaseForThisFile, isCorrectedRebaseForThisFile,
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.
*
* The function assumes all elements in read schema are included in file schema, so please
* run this check after clipping read schema upon file schema.
*
* 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,
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
struct.fields.foreach { f =>
val curFile = fileFieldMap(
if (isCaseSensitive) f.name else f.name.toLowerCase(Locale.ROOT))
checkSchemaCompat(curFile,
f.dataType,
errorCallback,
isCaseSensitive,
// Record root types for each column, so as to throw a readable exception
// over nested types.
Some(rootFileType.getOrElse(curFile)),
Some(rootReadType.getOrElse(f.dataType)))
}
case array: ArrayType =>
val fileChild = fileType.asGroupType().getType(0)
.asGroupType().getType(0)
checkSchemaCompat(fileChild, array.elementType, errorCallback, isCaseSensitive,
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,
rootFileType, rootReadType)
checkSchemaCompat(parquetMapValue, map.valueType, errorCallback, isCaseSensitive,
rootFileType, rootReadType)
case dt =>
checkPrimitiveCompat(fileType.asPrimitiveType(),
dt,
() => errorCallback(rootFileType.get, rootReadType.get))
}
}
/**
* 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 ||
dt == DataTypes.BinaryType || canReadAsBinaryDecimal(pt, dt) =>
return
case PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY if canReadAsIntDecimal(pt, dt) ||
canReadAsLongDecimal(pt, dt) || canReadAsBinaryDecimal(pt, dt) =>
return
case _ =>
}
// If we get here, it means the combination of Spark and Parquet type is invalid or not
// supported.
errorCallback()
}
private def throwTypeIncompatibleError(parquetType: Type,
sparkType: DataType,
filePath: String): Unit = {
val exception = new SchemaColumnConvertNotSupportedException(
parquetType.getName,
parquetType.toString,
sparkType.catalogString)
// A copy of QueryExecutionErrors.unsupportedSchemaColumnConvertError introduced in 3.2+
// TODO: replace with unsupportedSchemaColumnConvertError after we deprecate Spark 3.1
val message = "Parquet column cannot be converted in " +
s"file $filePath. Column: ${parquetType.getName}, " +
s"Expected: ${sparkType.catalogString}, Found: $parquetType"
throw new QueryExecutionException(message, exception)
}
private def isLongDecimal(dt: DataType): Boolean =
dt match {
case d: DecimalType => d.precision == 20 && d.scale == 0
case _ => false
}
// TODO: After we deprecate Spark 3.1, fetch decimal meta with DecimalLogicalTypeAnnotation
@scala.annotation.nowarn("msg=method getDecimalMetadata in class PrimitiveType is deprecated")
private def canReadAsIntDecimal(pt: PrimitiveType, dt: DataType) = {
DecimalType.is32BitDecimalType(dt) && isDecimalTypeMatched(pt.getDecimalMetadata, dt)
}
// TODO: After we deprecate Spark 3.1, fetch decimal meta with DecimalLogicalTypeAnnotation
@scala.annotation.nowarn("msg=method getDecimalMetadata in class PrimitiveType is deprecated")
private def canReadAsLongDecimal(pt: PrimitiveType, dt: DataType): Boolean = {
DecimalType.is64BitDecimalType(dt) && isDecimalTypeMatched(pt.getDecimalMetadata, dt)
}
// TODO: After we deprecate Spark 3.1, fetch decimal meta with DecimalLogicalTypeAnnotation
@scala.annotation.nowarn("msg=method getDecimalMetadata in class PrimitiveType is deprecated")
private def canReadAsBinaryDecimal(pt: PrimitiveType, dt: DataType): Boolean = {
DecimalType.isByteArrayDecimalType(dt) && isDecimalTypeMatched(pt.getDecimalMetadata, dt)
}
// TODO: After we deprecate Spark 3.1, fetch decimal meta with DecimalLogicalTypeAnnotation
@scala.annotation.nowarn("msg=class DecimalMetadata in package schema is deprecated")
private def isDecimalTypeMatched(metadata: DecimalMetadata,
sparkType: DataType): Boolean = {
if (metadata == null) {
false
} else {
val dt = sparkType.asInstanceOf[DecimalType]
metadata.getPrecision <= dt.precision && metadata.getScale == dt.scale
}
}
}
/**
* Similar to GpuParquetPartitionReaderFactory but extended for reading multiple files
* in an iteration. This will allow us to read multiple small files and combine them
* on the CPU side before sending them down to the GPU.
*/
case class GpuParquetMultiFilePartitionReaderFactory(
@transient sqlConf: SQLConf,
broadcastedConf: Broadcast[SerializableConfiguration],
dataSchema: StructType,
readDataSchema: StructType,
partitionSchema: StructType,
filters: Array[Filter],
@transient rapidsConf: RapidsConf,
metrics: Map[String, GpuMetric],
queryUsesInputFile: Boolean)
extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, rapidsConf) {
private val isCaseSensitive = sqlConf.caseSensitiveAnalysis
private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix
private val numThreads = rapidsConf.parquetMultiThreadReadNumThreads
private val maxNumFileProcessed = rapidsConf.maxNumParquetFilesParallel
private val footerReadType = rapidsConf.parquetReaderFooterType
private val ignoreMissingFiles = sqlConf.ignoreMissingFiles
private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
private val filterHandler = GpuParquetFileFilterHandler(sqlConf)
// we can't use the coalescing files reader when InputFileName, InputFileBlockStart,
// or InputFileBlockLength because we are combining all the files into a single buffer
// and we don't know which file is associated with each row.
override val canUseCoalesceFilesReader: Boolean =
rapidsConf.isParquetCoalesceFileReadEnabled && !(queryUsesInputFile || ignoreCorruptFiles)
override val canUseMultiThreadReader: Boolean = rapidsConf.isParquetMultiThreadReadEnabled
/**
* Build the PartitionReader for cloud reading
*
* @param files files to be read
* @param conf configuration
* @return cloud reading PartitionReader
*/
override def buildBaseColumnarReaderForCloud(
files: Array[PartitionedFile],
conf: Configuration): PartitionReader[ColumnarBatch] = {
new MultiFileCloudParquetPartitionReader(conf, footerReadType, files,
isCaseSensitive, readDataSchema, debugDumpPrefix,
maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, partitionSchema,
numThreads, maxNumFileProcessed, filterHandler, filters,
ignoreMissingFiles, ignoreCorruptFiles)
}
/**
* Build the PartitionReader for coalescing reading
*
* @param files files to be read
* @param conf the configuration
* @return coalescing reading PartitionReader
*/
override def buildBaseColumnarReaderForCoalescing(
files: Array[PartitionedFile],
conf: Configuration): PartitionReader[ColumnarBatch] = {
val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]()
files.map { file =>
val singleFileInfo = try {
filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema)
} catch {
case e: FileNotFoundException if ignoreMissingFiles =>
logWarning(s"Skipped missing file: ${file.filePath}", e)
ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty,
file.partitionValues, null, false, false, false)
// Throw FileNotFoundException even if `ignoreCorruptFiles` is true
case e: FileNotFoundException if !ignoreMissingFiles => throw e
// If ignoreMissingFiles=true, this case will never be reached. But it's ok
// to leave this branch here.
case e@(_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
logWarning(
s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e)
ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty,
file.partitionValues, null, false, false, false)
}
clippedBlocks ++= singleFileInfo.blocks.map(block =>
ParquetSingleDataBlockMeta(
singleFileInfo.filePath,