-
Notifications
You must be signed in to change notification settings - Fork 232
/
GpuColumnarBatchSerializer.scala
246 lines (216 loc) · 8.45 KB
/
GpuColumnarBatchSerializer.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
/*
* Copyright (c) 2019-2020, 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._
import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
import ai.rapids.cudf.{HostColumnVector, JCudfSerialization, NvtxColor, NvtxRange}
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import org.apache.spark.TaskContext
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance}
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.vectorized.ColumnarBatch
/**
* Serializer for serializing `ColumnarBatch`s during shuffle.
* The batches will be stored in an internal format specific to rapids.
*/
class GpuColumnarBatchSerializer(
numFields: Int,
dataSize: SQLMetric = null) extends Serializer with Serializable {
override def newInstance(): SerializerInstance =
new GpuColumnarBatchSerializerInstance(numFields, dataSize)
override def supportsRelocationOfSerializedObjects: Boolean = true
}
private class GpuColumnarBatchSerializerInstance(
numFields: Int,
dataSize: SQLMetric) extends SerializerInstance {
override def serializeStream(out: OutputStream): SerializationStream = new SerializationStream {
private[this] val dOut: DataOutputStream =
new DataOutputStream(new BufferedOutputStream(out))
override def writeValue[T: ClassTag](value: T): SerializationStream = {
val batch = value.asInstanceOf[ColumnarBatch]
val numColumns = batch.numCols()
val columns: Array[HostColumnVector] = new Array(numColumns)
val toClose = new ArrayBuffer[AutoCloseable]()
try {
var startRow = 0
val numRows = batch.numRows()
if (batch.numCols() > 0) {
val firstCol = batch.column(0)
if (firstCol.isInstanceOf[SlicedGpuColumnVector]) {
// We don't have control over ColumnarBatch to put in the slice, so we have to do it
// for each column. In this case we are using the first column.
startRow = firstCol.asInstanceOf[SlicedGpuColumnVector].getStart
for (i <- 0 until numColumns) {
columns(i) = batch.column(i).asInstanceOf[SlicedGpuColumnVector].getBase
}
} else {
for (i <- 0 until numColumns) {
batch.column(i) match {
case gpu: GpuColumnVector =>
val cpu = gpu.copyToHost()
toClose += cpu
columns(i) = cpu.getBase
case cpu: RapidsHostColumnVector =>
columns(i) = cpu.getBase
}
}
}
if (dataSize != null) {
dataSize.add(JCudfSerialization.getSerializedSizeInBytes(columns, startRow, numRows))
}
val range = new NvtxRange("Serialize Batch", NvtxColor.YELLOW)
try {
JCudfSerialization.writeToStream(columns, dOut, startRow, numRows)
} finally {
range.close()
}
} else {
val range = new NvtxRange("Serialize Row Only Batch", NvtxColor.YELLOW)
try {
JCudfSerialization.writeRowsToStream(dOut, numRows)
} finally {
range.close()
}
}
} finally {
toClose.safeClose()
}
this
}
override def writeKey[T: ClassTag](key: T): SerializationStream = {
// The key is only needed on the map side when computing partition ids. It does not need to
// be shuffled.
assert(null == key || key.isInstanceOf[Int])
this
}
override def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = {
// This method is never called by shuffle code.
throw new UnsupportedOperationException
}
override def writeObject[T: ClassTag](t: T): SerializationStream = {
// This method is never called by shuffle code.
throw new UnsupportedOperationException
}
override def flush(): Unit = {
dOut.flush()
}
override def close(): Unit = {
dOut.close()
}
}
override def deserializeStream(in: InputStream): DeserializationStream = {
new DeserializationStream {
private[this] val dIn: DataInputStream = new DataInputStream(new BufferedInputStream(in))
override def asKeyValueIterator: Iterator[(Int, ColumnarBatch)] = {
new Iterator[(Int, ColumnarBatch)] {
var toBeReturned: Option[ColumnarBatch] = None
TaskContext.get().addTaskCompletionListener[Unit]((tc: TaskContext) => {
toBeReturned.foreach(_.close())
toBeReturned = None
dIn.close()
})
def tryReadNext(): Option[ColumnarBatch] = {
// about to start using the GPU in this task
GpuSemaphore.acquireIfNecessary(TaskContext.get())
val range = new NvtxRange("Deserialize Batch", NvtxColor.YELLOW)
try {
val tableInfo = JCudfSerialization.readTableFrom(dIn)
try {
val table = tableInfo.getTable
if (table == null && tableInfo.getNumRows == 0) {
dIn.close()
None
} else {
if (table != null) {
Some(GpuColumnVector.from(table))
} else {
Some(new ColumnarBatch(Array.empty, tableInfo.getNumRows))
}
}
} finally {
tableInfo.close()
}
} finally {
range.close()
}
}
override def hasNext: Boolean = {
if (toBeReturned.isEmpty) {
toBeReturned = tryReadNext()
}
toBeReturned.isDefined
}
override def next(): (Int, ColumnarBatch) = {
if (toBeReturned.isEmpty) {
toBeReturned = tryReadNext()
if (toBeReturned.isEmpty) {
throw new NoSuchElementException("Walked off of the end...")
}
}
val ret = toBeReturned.get
toBeReturned = None
(0, ret)
}
}
}
override def asIterator: Iterator[Any] = {
// This method is never called by shuffle code.
throw new UnsupportedOperationException
}
override def readKey[T]()(implicit classType: ClassTag[T]): T = {
// We skipped serialization of the key in writeKey(), so just return a dummy value since
// this is going to be discarded anyways.
null.asInstanceOf[T]
}
override def readValue[T]()(implicit classType: ClassTag[T]): T = {
// about to start using the GPU in this task
GpuSemaphore.acquireIfNecessary(TaskContext.get())
val range = new NvtxRange("Deserialize Batch", NvtxColor.YELLOW)
try {
val tableInfo = JCudfSerialization.readTableFrom(dIn)
val cb = try {
val table = tableInfo.getTable
if (table != null) {
Some(GpuColumnVector.from(table))
} else {
Some(new ColumnarBatch(Array.empty, tableInfo.getNumRows))
}
} finally {
tableInfo.close()
}
cb.asInstanceOf[T]
} finally {
range.close()
}
}
override def readObject[T]()(implicit classType: ClassTag[T]): T = {
// This method is never called by shuffle code.
throw new UnsupportedOperationException
}
override def close(): Unit = {
dIn.close()
}
}
}
// These methods are never called by shuffle code.
override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException
override def deserialize[T: ClassTag](bytes: ByteBuffer): T =
throw new UnsupportedOperationException
override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T =
throw new UnsupportedOperationException
}