-
Notifications
You must be signed in to change notification settings - Fork 232
/
ParquetCachedBatchSerializer.scala
1541 lines (1370 loc) · 59.6 KB
/
ParquetCachedBatchSerializer.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) 2020-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.shims
import java.io.{InputStream, IOException}
import java.lang.reflect.Method
import java.nio.ByteBuffer
import java.util.concurrent.atomic.AtomicLong
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import ai.rapids.cudf._
import ai.rapids.cudf.ParquetWriterOptions.StatisticsFrequency
import com.nvidia.spark.GpuCachedBatchSerializer
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import java.util
import org.apache.commons.io.output.ByteArrayOutputStream
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.RecordWriter
import org.apache.parquet.{HadoopReadOptions, ParquetReadOptions}
import org.apache.parquet.column.{ColumnDescriptor, ParquetProperties}
import org.apache.parquet.hadoop.{CodecFactory, MemoryManager, ParquetFileReader, ParquetFileWriter, ParquetInputFormat, ParquetOutputFormat, ParquetRecordWriter, ParquetWriter}
import org.apache.parquet.hadoop.ParquetFileWriter.Mode
import org.apache.parquet.hadoop.api.WriteSupport
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.parquet.io.{DelegatingPositionOutputStream, DelegatingSeekableInputStream, InputFile, OutputFile, PositionOutputStream, SeekableInputStream}
import org.apache.parquet.schema.Type
import org.apache.spark.TaskContext
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{vectorized, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericInternalRow, SpecializedGetters, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
import org.apache.spark.sql.columnar.CachedBatch
import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetToSparkSchemaConverter, ParquetWriteSupport, VectorizedColumnReader}
import org.apache.spark.sql.execution.datasources.parquet.rapids.shims.{ParquetRecordMaterializer, ShimVectorizedColumnReader}
import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.storage.StorageLevel
import org.apache.spark.unsafe.types.CalendarInterval
/**
* copied from Spark org.apache.spark.util.ByteBufferInputStream
*/
private class ByteBufferInputStream(private var buffer: ByteBuffer)
extends InputStream {
override def read(): Int = {
if (buffer == null || buffer.remaining() == 0) {
cleanUp()
-1
} else {
buffer.get() & 0xFF
}
}
override def read(dest: Array[Byte]): Int = {
read(dest, 0, dest.length)
}
override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
if (buffer == null || buffer.remaining() == 0) {
cleanUp()
-1
} else {
val amountToGet = math.min(buffer.remaining(), length)
buffer.get(dest, offset, amountToGet)
amountToGet
}
}
override def skip(bytes: Long): Long = {
if (buffer != null) {
val amountToSkip = math.min(bytes, buffer.remaining).toInt
buffer.position(buffer.position() + amountToSkip)
if (buffer.remaining() == 0) {
cleanUp()
}
amountToSkip
} else {
0L
}
}
/**
* Clean up the buffer, and potentially dispose of it using StorageUtils.dispose().
*/
private def cleanUp(): Unit = {
if (buffer != null) {
buffer = null
}
}
}
class ByteArrayInputFile(buff: Array[Byte]) extends InputFile {
override def getLength: Long = buff.length
override def newStream(): SeekableInputStream = {
val byteBuffer = ByteBuffer.wrap(buff)
new DelegatingSeekableInputStream(new ByteBufferInputStream(byteBuffer)) {
override def getPos: Long = byteBuffer.position()
override def seek(newPos: Long): Unit = {
if (newPos > Int.MaxValue || newPos < Int.MinValue) {
throw new IllegalStateException("seek value is out of supported range " + newPos)
}
byteBuffer.position(newPos.toInt)
}
}
}
}
private object ByteArrayOutputFile {
val BLOCK_SIZE: Int = 32 * 1024 * 1024 // 32M
}
private class ByteArrayOutputFile(stream: ByteArrayOutputStream) extends OutputFile {
override def create(blockSizeHint: Long): PositionOutputStream = {
new DelegatingPositionOutputStream(stream) {
var pos = 0
override def getPos: Long = pos
override def write(b: Int): Unit = {
super.write(b)
pos += Integer.BYTES
}
override def write(b: Array[Byte]): Unit = {
super.write(b)
pos += b.length
}
override def write(b: Array[Byte], off: Int, len: Int): Unit = {
super.write(b, off, len)
pos += len
}
}
}
override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream =
throw new UnsupportedOperationException("Don't need to overwrite")
override def supportsBlockSize(): Boolean = true
override def defaultBlockSize(): Long = ByteArrayOutputFile.BLOCK_SIZE
}
private class ParquetBufferConsumer(val numRows: Int) extends HostBufferConsumer with
AutoCloseable {
@transient private[this] val offHeapBuffers = mutable.Queue[(HostMemoryBuffer, Long)]()
private var buffer: Array[Byte] = _
override def handleBuffer(buffer: HostMemoryBuffer, len: Long): Unit = {
offHeapBuffers += Tuple2(buffer, len)
}
def getBuffer: Array[Byte] = {
if (buffer == null) {
writeBuffers()
}
buffer
}
def close(): Unit = {
if (buffer == null) {
writeBuffers()
}
}
private def writeBuffers(): Unit = {
val toProcess = offHeapBuffers.dequeueAll(_ => true)
// We are making sure the input is smaller than 2gb so the parquet written should never be more
// than Int.MAX_SIZE.
val bytes = toProcess.map(_._2).sum
// for now assert bytes are less than Int.MaxValue
assert(bytes <= Int.MaxValue)
buffer = new Array(bytes.toInt)
try {
var offset: Int = 0
toProcess.foreach(ops => {
val origBuffer = ops._1
val len = ops._2.toInt
origBuffer.asByteBuffer().get(buffer, offset, len)
offset = offset + len
})
} finally {
toProcess.map(_._1).safeClose()
}
}
}
private object ParquetCachedBatch {
def apply(parquetBuff: ParquetBufferConsumer): ParquetCachedBatch = {
new ParquetCachedBatch(parquetBuff.numRows, parquetBuff.getBuffer)
}
}
case class ParquetCachedBatch(
numRows: Int,
buffer: Array[Byte]) extends CachedBatch {
override def sizeInBytes: Long = buffer.length
}
/**
* Spark wants the producer to close the batch. We have a listener in this iterator that will close
* the batch after the task is completed
*/
private case class CloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]) extends
Iterator[ColumnarBatch] {
var cb: ColumnarBatch = _
private def closeCurrentBatch(): Unit = {
if (cb != null) {
cb.close()
cb = null
}
}
TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => {
closeCurrentBatch()
})
override def hasNext: Boolean = iter.hasNext
override def next(): ColumnarBatch = {
closeCurrentBatch()
cb = iter.next()
cb
}
}
/**
* This class assumes, the data is Columnar and the plugin is on.
* Note, this class should not be referenced directly in source code.
* It should be loaded by reflection using ShimLoader.newInstanceOf, see ./docs/dev/shims.md
*/
protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer with Arm {
override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = true
override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall { f =>
// only check spark b/c if we are on the GPU then we will be calling the gpu method regardless
isTypeSupportedByColumnarSparkParquetWriter(f.dataType) || f.dataType == DataTypes.NullType
}
private def isTypeSupportedByColumnarSparkParquetWriter(dataType: DataType): Boolean = {
// Columnar writer in Spark only supports AtomicTypes ATM
dataType match {
case TimestampType | StringType | BooleanType | DateType | BinaryType |
DoubleType | FloatType | ByteType | IntegerType | LongType | ShortType => true
case _: DecimalType => true
case _ => false
}
}
def isSchemaSupportedByCudf(schema: Seq[Attribute]): Boolean = {
schema.forall(field => isSupportedByCudf(field.dataType))
}
def isSupportedByCudf(dataType: DataType): Boolean = {
dataType match {
case a: ArrayType => isSupportedByCudf(a.elementType)
case s: StructType => s.forall(field => isSupportedByCudf(field.dataType))
case m: MapType => isSupportedByCudf(m.keyType) && isSupportedByCudf(m.valueType)
case _ => GpuColumnVector.isNonNestedSupportedType(dataType)
}
}
/**
* This method checks if the datatype passed is officially supported by parquet.
*
* Please refer to https://github.com/apache/parquet-format/blob/master/LogicalTypes.md to see
* the what types are supported by parquet
*/
def isTypeSupportedByParquet(dataType: DataType): Boolean = {
dataType match {
case CalendarIntervalType | NullType => false
case s: StructType => s.forall(field => isTypeSupportedByParquet(field.dataType))
case ArrayType(elementType, _) => isTypeSupportedByParquet(elementType)
case MapType(keyType, valueType, _) => isTypeSupportedByParquet(keyType) &&
isTypeSupportedByParquet(valueType)
case d: DecimalType if d.scale < 0 => false
case _ => true
}
}
/**
* Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data.
* This method uses Parquet Writer on the GPU to write the cached batch
*
* @param input the input `RDD` to be converted.
* @param schema the schema of the data being stored.
* @param storageLevel where the data will be stored.
* @param conf the config for the query.
* @return The data converted into a format more suitable for caching.
*/
override def convertColumnarBatchToCachedBatch(input: RDD[ColumnarBatch],
schema: Seq[Attribute],
storageLevel: StorageLevel,
conf: SQLConf): RDD[CachedBatch] = {
val rapidsConf = new RapidsConf(conf)
val bytesAllowedPerBatch = getBytesAllowedPerBatch(conf)
val (schemaWithUnambiguousNames, _) = getSupportedSchemaFromUnsupported(schema)
val structSchema = schemaWithUnambiguousNames.toStructType
if (rapidsConf.isSqlEnabled && rapidsConf.isSqlExecuteOnGPU &&
isSchemaSupportedByCudf(schema)) {
def putOnGpuIfNeeded(batch: ColumnarBatch): ColumnarBatch = {
if (!batch.column(0).isInstanceOf[GpuColumnVector]) {
val s: StructType = structSchema
val gpuCB = new GpuColumnarBatchBuilder(s, batch.numRows()).build(batch.numRows())
batch.close()
gpuCB
} else {
batch
}
}
input.flatMap(batch => {
if (batch.numCols() == 0) {
List(ParquetCachedBatch(batch.numRows(), new Array[Byte](0)))
} else {
withResource(putOnGpuIfNeeded(batch)) { gpuCB =>
compressColumnarBatchWithParquet(gpuCB, structSchema, schema.toStructType,
bytesAllowedPerBatch)
}
}
})
} else {
val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
input.mapPartitions {
cbIter =>
new CachedBatchIteratorProducer[ColumnarBatch](cbIter, schemaWithUnambiguousNames, schema,
broadcastedConf).getColumnarBatchToCachedBatchIterator
}
}
}
private[rapids] def compressColumnarBatchWithParquet(
oldGpuCB: ColumnarBatch,
schema: StructType,
origSchema: StructType,
bytesAllowedPerBatch: Long): List[ParquetCachedBatch] = {
val estimatedRowSize = scala.Range(0, oldGpuCB.numCols()).map { idx =>
oldGpuCB.column(idx).asInstanceOf[GpuColumnVector]
.getBase.getDeviceMemorySize / oldGpuCB.numRows()
}.sum
val columns = for (i <- 0 until oldGpuCB.numCols()) yield {
val gpuVector = oldGpuCB.column(i).asInstanceOf[GpuColumnVector]
var dataType = origSchema(i).dataType
val v = ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(gpuVector.getBase,
origSchema(i).dataType,
// we are checking for scale > 0 because cudf and spark refer to scales as opposites
// e.g. scale = -3 in Spark is scale = 3 in cudf
(_, cv) => cv.getType.isDecimalType && cv.getType.getScale > 0,
(_, cv) => {
if (cv.getType.isBackedByLong) {
dataType = LongType
cv.bitCastTo(DType.INT64)
} else {
dataType = IntegerType
cv.bitCastTo(DType.INT32)
}
}
)
GpuColumnVector.from(v, schema(i).dataType)
}
withResource(new ColumnarBatch(columns.toArray, oldGpuCB.numRows())) { gpuCB =>
val rowsAllowedInBatch = (bytesAllowedPerBatch / estimatedRowSize).toInt
val splitIndices = scala.Range(rowsAllowedInBatch, gpuCB.numRows(), rowsAllowedInBatch)
val buffers = new ListBuffer[ParquetCachedBatch]
if (splitIndices.nonEmpty) {
val splitVectors = new ListBuffer[Array[ColumnVector]]
try {
for (index <- 0 until gpuCB.numCols()) {
splitVectors +=
gpuCB.column(index).asInstanceOf[GpuColumnVector].getBase.split(splitIndices: _*)
}
// Splitting the table
// e.g. T0 = {col1, col2,...,coln} => split columns into 'm' cols =>
// T00= {splitCol1(0), splitCol2(0),...,splitColn(0)}
// T01= {splitCol1(1), splitCol2(1),...,splitColn(1)}
// ...
// T0m= {splitCol1(m), splitCol2(m),...,splitColn(m)}
def makeTableForIndex(i: Int): Table = {
val columns = splitVectors.indices.map(j => splitVectors(j)(i))
new Table(columns: _*)
}
for (i <- splitVectors.head.indices) {
withResource(makeTableForIndex(i)) { table =>
val buffer = writeTableToCachedBatch(table, schema)
buffers += ParquetCachedBatch(buffer)
}
}
} finally {
splitVectors.foreach(array => array.safeClose())
}
} else {
withResource(GpuColumnVector.from(gpuCB)) { table =>
val buffer = writeTableToCachedBatch(table, schema)
buffers += ParquetCachedBatch(buffer)
}
}
buffers.toList
}
}
private def writeTableToCachedBatch(
table: Table,
schema: StructType): ParquetBufferConsumer = {
val buffer = new ParquetBufferConsumer(table.getRowCount.toInt)
val opts = SchemaUtils
.writerOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false)
.withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build()
withResource(Table.writeParquetChunked(opts, buffer)) { writer =>
writer.write(table)
}
buffer
}
/**
* This method decodes the CachedBatch leaving it on the GPU to avoid the extra copying back to
* the host
*
* @param input the cached batches that should be converted.
* @param cacheAttributes the attributes of the data in the batch.
* @param selectedAttributes the fields that should be loaded from the data and the order they
* should appear in the output batch.
* @param conf the configuration for the job.
* @return an RDD of the input cached batches transformed into the ColumnarBatch format.
*/
def gpuConvertCachedBatchToColumnarBatch(
input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[ColumnarBatch] = {
// optimize
val newSelectedAttributes = if (selectedAttributes.isEmpty) {
cacheAttributes
} else {
selectedAttributes
}
val (cachedSchemaWithNames, selectedSchemaWithNames) =
getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
convertCachedBatchToColumnarInternal(
input,
cachedSchemaWithNames,
selectedSchemaWithNames,
newSelectedAttributes)
}
private def convertCachedBatchToColumnarInternal(
input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
originalSelectedAttributes: Seq[Attribute]): RDD[ColumnarBatch] = {
val cbRdd: RDD[ColumnarBatch] = input.map {
case parquetCB: ParquetCachedBatch =>
val parquetOptions = ParquetOptions.builder()
.includeColumn(selectedAttributes.map(_.name).asJavaCollection).build()
withResource(Table.readParquet(parquetOptions, parquetCB.buffer, 0,
parquetCB.sizeInBytes)) { table =>
withResource {
for (i <- 0 until table.getNumberOfColumns) yield {
ColumnCastUtil.ifTrueThenDeepConvertTypeAtoTypeB(table.getColumn(i),
originalSelectedAttributes(i).dataType,
(dataType, _) => dataType match {
case d: DecimalType if d.scale < 0 => true
case _ => false
},
(dataType, cv) => {
dataType match {
case d: DecimalType =>
withResource(cv.bitCastTo(DecimalUtil.createCudfDecimal(d))) {
_.copyToColumnVector()
}
case _ =>
throw new IllegalStateException("We don't cast any type besides Decimal " +
"with scale < 0")
}
}
)
}
} { col =>
withResource(new Table(col: _*)) { t =>
GpuColumnVector.from(t, originalSelectedAttributes.map(_.dataType).toArray)
}
}
}
case _ =>
throw new IllegalStateException("I don't know how to convert this batch")
}
cbRdd
}
private def getSelectedSchemaFromCachedSchema(
selectedAttributes: Seq[Attribute],
cacheAttributes: Seq[Attribute]): Seq[Attribute] = {
selectedAttributes.map {
a => cacheAttributes(cacheAttributes.map(_.exprId).indexOf(a.exprId))
}
}
/**
* Convert the cached data into a ColumnarBatch taking the result data back to the host
*
* @param input the cached batches that should be converted.
* @param cacheAttributes the attributes of the data in the batch.
* @param selectedAttributes the fields that should be loaded from the data and the order they
* should appear in the output batch.
* @param conf the configuration for the job.
* @return an RDD of the input cached batches transformed into the ColumnarBatch format.
*/
override def convertCachedBatchToColumnarBatch(input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[ColumnarBatch] = {
// optimize
val newSelectedAttributes = if (selectedAttributes.isEmpty) {
cacheAttributes
} else {
selectedAttributes
}
val rapidsConf = new RapidsConf(conf)
val (cachedSchemaWithNames, selectedSchemaWithNames) =
getSupportedSchemaFromUnsupported(cacheAttributes, newSelectedAttributes)
if (rapidsConf.isSqlEnabled && rapidsConf.isSqlExecuteOnGPU &&
isSchemaSupportedByCudf(cachedSchemaWithNames)) {
val batches = convertCachedBatchToColumnarInternal(input, cachedSchemaWithNames,
selectedSchemaWithNames, newSelectedAttributes)
val cbRdd = batches.map(batch => {
withResource(batch) { gpuBatch =>
val cols = GpuColumnVector.extractColumns(gpuBatch)
new ColumnarBatch(cols.safeMap(_.copyToHost()).toArray, gpuBatch.numRows())
}
})
cbRdd.mapPartitions(iter => CloseableColumnBatchIterator(iter))
} else {
val origSelectedAttributesWithUnambiguousNames =
sanitizeColumnNames(newSelectedAttributes, selectedSchemaWithNames)
val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
input.mapPartitions {
cbIter => {
new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
cacheAttributes, origSelectedAttributesWithUnambiguousNames, broadcastedConf)
.getColumnBatchIterator
}
}
}
}
/**
* Convert the cached batch into `InternalRow`s.
*
* @param input the cached batches that should be converted.
* @param cacheAttributes the attributes of the data in the batch.
* @param selectedAttributes the field that should be loaded from the data and the order they
* should appear in the output rows.
* @param conf the configuration for the job.
* @return RDD of the rows that were stored in the cached batches.
*/
override def convertCachedBatchToInternalRow(
input: RDD[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
conf: SQLConf): RDD[InternalRow] = {
val (cachedSchemaWithNames, selectedSchemaWithNames) =
getSupportedSchemaFromUnsupported(cacheAttributes, selectedAttributes)
val newSelectedAttributes = sanitizeColumnNames(selectedAttributes, selectedSchemaWithNames)
val broadcastedConf = SparkSession.active.sparkContext.broadcast(conf.getAllConfs)
input.mapPartitions {
cbIter => {
new CachedBatchIteratorConsumer(cbIter, cachedSchemaWithNames, selectedSchemaWithNames,
cacheAttributes, newSelectedAttributes, broadcastedConf).getInternalRowIterator
}
}
}
private abstract class UnsupportedDataHandlerIterator extends Iterator[InternalRow] {
def handleInternalRow(schema: Seq[Attribute], row: InternalRow, newRow: InternalRow): Unit
def handleInterval(data: SpecializedGetters, index: Int): Any
def handleStruct(
data: InternalRow,
origSchema: StructType,
supportedSchema: StructType): InternalRow = {
val structRow = InternalRow.fromSeq(supportedSchema)
handleInternalRow(origSchema.map(field =>
AttributeReference(field.name, field.dataType, field.nullable)()), data, structRow)
structRow
}
def handleMap(
keyType: DataType,
valueType: DataType,
mapData: MapData): MapData = {
val keyData = mapData.keyArray()
val newKeyData = handleArray(keyType, keyData)
val valueData = mapData.valueArray()
val newValueData = handleArray(valueType, valueData)
new ArrayBasedMapData(newKeyData, newValueData)
}
def handleArray(
dataType: DataType,
arrayData: ArrayData): ArrayData = {
dataType match {
case s@StructType(_) =>
val listBuffer = new ListBuffer[InternalRow]()
val supportedSchema = mapping(dataType).asInstanceOf[StructType]
arrayData.foreach(supportedSchema, (_, data) => {
val structRow =
handleStruct(data.asInstanceOf[InternalRow], s, s)
listBuffer += structRow.copy()
})
new GenericArrayData(listBuffer)
case ArrayType(elementType, _) =>
val arrayList = new ListBuffer[Any]()
scala.Range(0, arrayData.numElements()).foreach { i =>
val subArrayData = arrayData.getArray(i)
arrayList.append(handleArray(elementType, subArrayData))
}
new GenericArrayData(arrayList)
case m@MapType(_, _, _) =>
val mapList =
new ListBuffer[Any]()
scala.Range(0, arrayData.numElements()).foreach { i =>
val mapData = arrayData.getMap(i)
mapList.append(handleMap(m.keyType, m.valueType, mapData))
}
new GenericArrayData(mapList)
case CalendarIntervalType =>
val citList = new ListBuffer[Any]()
scala.Range(0, arrayData.numElements()).foreach { i =>
val citRow = handleInterval(arrayData, i)
citList += citRow
}
new GenericArrayData(citList)
case _ =>
arrayData
}
}
}
/**
* Consumes the Iterator[CachedBatch] to return either Iterator[ColumnarBatch] or
* Iterator[InternalRow]
*/
private class CachedBatchIteratorConsumer(
cbIter: Iterator[CachedBatch],
cacheAttributes: Seq[Attribute],
selectedAttributes: Seq[Attribute],
origCacheSchema: Seq[Attribute],
origRequestedSchema: Seq[Attribute],
sharedConf: Broadcast[Map[String, String]]) {
val conf: SQLConf = getConfFromMap(sharedConf)
val hadoopConf: Configuration = getHadoopConf(origRequestedSchema.toStructType, conf)
val options: ParquetReadOptions = HadoopReadOptions.builder(hadoopConf).build()
/**
* We are getting this method using reflection because its a package-private
*/
val readBatchMethod: Method =
classOf[VectorizedColumnReader].getDeclaredMethod("readBatch", Integer.TYPE,
classOf[WritableColumnVector])
readBatchMethod.setAccessible(true)
def getInternalRowIterator: Iterator[InternalRow] = {
/**
* This iterator converts an iterator[CachedBatch] to an iterator[InternalRow].
*
* This makes it unlike a regular iterator because CachedBatch => InternalRow* is a 1-n
* relation. The way we have implemented this is to first go through the
* iterator[CachedBatch] (cbIter) to look for a valid iterator (iter) i.e. hasNext() => true.
* Then every time next() is called we return a single InternalRow from iter. When
* iter.hasNext() => false, we find the next valid iterator in cbIter and the process
* continues as above.
*/
new Iterator[InternalRow]() {
var iter: Iterator[InternalRow] = _
override def hasNext: Boolean = {
// go over the batch and get the next non-degenerate iterator
// and return if it hasNext
while ((iter == null || !iter.hasNext) && cbIter.hasNext) {
iter = convertCachedBatchToInternalRowIter
}
iter != null && iter.hasNext
}
override def next(): InternalRow = {
// will return the next InternalRow if hasNext() is true, otherwise throw
if (hasNext) {
iter.next()
} else {
throw new NoSuchElementException("no elements found")
}
}
/**
* This method converts a CachedBatch to an iterator of InternalRows.
*/
private def convertCachedBatchToInternalRowIter: Iterator[InternalRow] = {
val parquetCachedBatch = cbIter.next().asInstanceOf[ParquetCachedBatch]
val inputFile = new ByteArrayInputFile(parquetCachedBatch.buffer)
withResource(ParquetFileReader.open(inputFile, options)) { parquetFileReader =>
val parquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
val hasUnsupportedType = origCacheSchema.exists { field =>
!isTypeSupportedByParquet(field.dataType)
}
val unsafeRows = new ArrayBuffer[InternalRow]
import org.apache.parquet.io.ColumnIOFactory
var pages = parquetFileReader.readNextRowGroup()
while (pages != null) {
val rows = pages.getRowCount
val columnIO = new ColumnIOFactory().getColumnIO(parquetSchema)
val recordReader =
columnIO.getRecordReader(pages, new ParquetRecordMaterializer(parquetSchema,
cacheAttributes.toStructType,
new ParquetToSparkSchemaConverter(hadoopConf), None /*convertTz*/ ,
LegacyBehaviorPolicy.CORRECTED))
for (_ <- 0 until rows.toInt) {
val row = recordReader.read
unsafeRows += row.copy()
}
pages = parquetFileReader.readNextRowGroup()
}
val iter = unsafeRows.iterator
val unsafeProjection =
GenerateUnsafeProjection.generate(selectedAttributes, cacheAttributes)
if (hasUnsupportedType) {
new UnsupportedDataHandlerIterator() {
val wrappedIter: Iterator[InternalRow] = iter
val newRow = new GenericInternalRow(cacheAttributes.length)
override def hasNext: Boolean = wrappedIter.hasNext
override def next(): InternalRow = {
//read a row and convert it to what the caller is expecting
val row = wrappedIter.next()
handleInternalRow(origCacheSchema, row, newRow)
val unsafeProjection =
GenerateUnsafeProjection.generate(origRequestedSchema, origCacheSchema)
unsafeProjection.apply(newRow)
}
override def handleInterval(
data: SpecializedGetters,
index: Int): CalendarInterval = {
if (data.isNullAt(index)) {
null
} else {
val structData = data.getStruct(index, 3)
new CalendarInterval(structData.getInt(0),
structData.getInt(1), structData.getLong(2))
}
}
override def handleInternalRow(
schema: Seq[Attribute],
row: InternalRow,
newRow: InternalRow): Unit = {
schema.indices.foreach { index =>
val dataType = schema(index).dataType
if (mapping.contains(dataType) || dataType == CalendarIntervalType ||
dataType == NullType ||
(dataType.isInstanceOf[DecimalType]
&& dataType.asInstanceOf[DecimalType].scale < 0)) {
if (row.isNullAt(index)) {
newRow.setNullAt(index)
} else {
dataType match {
case s@StructType(_) =>
val supportedSchema = mapping(dataType)
.asInstanceOf[StructType]
val structRow =
handleStruct(row.getStruct(index, supportedSchema.size), s, s)
newRow.update(index, structRow)
case a@ArrayType(_, _) =>
val arrayData = row.getArray(index)
newRow.update(index, handleArray(a.elementType, arrayData))
case MapType(keyType, valueType, _) =>
val mapData = row.getMap(index)
newRow.update(index, handleMap(keyType, valueType, mapData))
case CalendarIntervalType =>
val interval = handleInterval(row, index)
if (interval == null) {
newRow.setNullAt(index)
} else {
newRow.setInterval(index, interval)
}
case d: DecimalType =>
if (row.isNullAt(index)) {
newRow.setDecimal(index, null, d.precision)
} else {
val dec = if (d.precision <= Decimal.MAX_INT_DIGITS) {
Decimal(row.getInt(index).toLong, d.precision, d.scale)
} else {
Decimal(row.getLong(index), d.precision, d.scale)
}
newRow.update(index, dec)
}
case NullType =>
newRow.setNullAt(index)
case _ =>
newRow.update(index, row.get(index, dataType))
}
}
} else {
newRow.update(index, row.get(index, dataType))
}
}
}
}
} else {
iter.map(unsafeProjection)
}
}
}
}
}
private class CurrentBatchIterator(val parquetCachedBatch: ParquetCachedBatch)
extends Iterator[ColumnarBatch] with AutoCloseable {
val capacity = conf.parquetVectorizedReaderBatchSize
var columnReaders: Array[VectorizedColumnReader] = _
val columnVectors: Array[OffHeapColumnVector] =
OffHeapColumnVector.allocateColumns(capacity, selectedAttributes.toStructType)
val columnarBatch = new ColumnarBatch(columnVectors
.asInstanceOf[Array[vectorized.ColumnVector]])
var rowsReturned: Long = 0L
var numBatched = 0
var batchIdx = 0
var totalCountLoadedSoFar: Long = 0
val parquetFileReader =
ParquetFileReader.open(new ByteArrayInputFile(parquetCachedBatch.buffer), options)
val (totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
columnsInCache, typesInCache) = {
val parquetToSparkSchemaConverter = new ParquetToSparkSchemaConverter(hadoopConf)
// we are getting parquet schema and then converting it to catalyst schema
// because catalyst schema that we get from Spark doesn't have the exact schema expected
// by the columnar parquet reader
val inMemCacheParquetSchema = parquetFileReader.getFooter.getFileMetaData.getSchema
val inMemCacheSparkSchema = parquetToSparkSchemaConverter.convert(inMemCacheParquetSchema)
val totalRowCount = parquetFileReader.getRowGroups.asScala.map(_.getRowCount).sum
val inMemReqSparkSchema = StructType(selectedAttributes.toStructType.map { field =>
inMemCacheSparkSchema.fields(inMemCacheSparkSchema.fieldIndex(field.name))
})
val reqSparkSchemaInCacheOrder = StructType(inMemCacheSparkSchema.filter(f =>
inMemReqSparkSchema.fields.exists(f0 => f0.name.equals(f.name))))
// There could be a case especially in a distributed environment where the requestedSchema
// and cacheSchema are not in the same order. We need to create a map so we can guarantee
// that we writing to the correct columnVector
val cacheSchemaToReqSchemaMap: Map[Int, Int] =
reqSparkSchemaInCacheOrder.indices.map { index =>
index -> inMemReqSparkSchema.fields.indexOf(reqSparkSchemaInCacheOrder.fields(index))
}.toMap
val reqParquetSchemaInCacheOrder = new org.apache.parquet.schema.MessageType(
inMemCacheParquetSchema.getName(), reqSparkSchemaInCacheOrder.fields.map { f =>
inMemCacheParquetSchema.getFields().get(inMemCacheParquetSchema.getFieldIndex(f.name))
}:_*)
val columnsRequested: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
// reset spark schema calculated from parquet schema
hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, inMemReqSparkSchema.json)
hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, inMemReqSparkSchema.json)
val columnsInCache: util.List[ColumnDescriptor] = reqParquetSchemaInCacheOrder.getColumns
val typesInCache: util.List[Type] = reqParquetSchemaInCacheOrder.asGroupType.getFields
val missingColumns = new Array[Boolean](reqParquetSchemaInCacheOrder.getFieldCount)
// initialize missingColumns to cover the case where requested column isn't present in the
// cache, which should never happen but just in case it does
val paths: util.List[Array[String]] = reqParquetSchemaInCacheOrder.getPaths
for (i <- 0 until reqParquetSchemaInCacheOrder.getFieldCount) {
val t = reqParquetSchemaInCacheOrder.getFields.get(i)
if (!t.isPrimitive || t.isRepetition(Type.Repetition.REPEATED)) {
throw new UnsupportedOperationException("Complex types not supported.")
}
val colPath = paths.get(i)
if (inMemCacheParquetSchema.containsPath(colPath)) {
val fd = inMemCacheParquetSchema.getColumnDescription(colPath)
if (!(fd == columnsRequested.get(i))) {
throw new UnsupportedOperationException("Schema evolution not supported.")
}
missingColumns(i) = false
} else {
if (columnsRequested.get(i).getMaxDefinitionLevel == 0) {
// Column is missing in data but the required data is non-nullable.
// This file is invalid.
throw new IOException(s"Required column is missing in data file: ${colPath.toList}")
}
missingColumns(i) = true
}
}
for (i <- missingColumns.indices) {
if (missingColumns(i)) {
columnVectors(i).putNulls(0, capacity)
columnVectors(i).setIsConstant()
}
}
(totalRowCount, columnsRequested, cacheSchemaToReqSchemaMap, missingColumns,
columnsInCache, typesInCache)
}
@throws[IOException]
def checkEndOfRowGroup(): Unit = {
if (rowsReturned != totalCountLoadedSoFar) return
val pages = parquetFileReader.readNextRowGroup
if (pages == null) {
throw new IOException("expecting more rows but reached last" +
" block. Read " + rowsReturned + " out of " + totalRowCount)
}
columnReaders = new Array[VectorizedColumnReader](columnsRequested.size)
for (i <- 0 until columnsRequested.size) {
if (!missingColumns(i)) {
columnReaders(i) =
new ShimVectorizedColumnReader(
i,
columnsInCache,
typesInCache,
pages,
null /*convertTz*/ ,
LegacyBehaviorPolicy.CORRECTED.toString,
LegacyBehaviorPolicy.EXCEPTION.toString, false)
}
}
totalCountLoadedSoFar += pages.getRowCount
}
/**
* Read the next RowGroup and read each column and return the columnarBatch
*/
def nextBatch: Boolean = {
for (vector <- columnVectors) {
vector.reset()
}
columnarBatch.setNumRows(0)
if (rowsReturned >= totalRowCount) return false
checkEndOfRowGroup()
val num = Math.min(capacity.toLong, totalCountLoadedSoFar - rowsReturned).toInt
for (i <- columnReaders.indices) {
if (columnReaders(i) != null) {
readBatchMethod.invoke(columnReaders(i), num.asInstanceOf[AnyRef],
columnVectors(cacheSchemaToReqSchemaMap(i)).asInstanceOf[AnyRef])
}
}
rowsReturned += num
columnarBatch.setNumRows(num)