forked from apache/spark
/
UnsafeGeneratedAggregate.scala
381 lines (330 loc) · 16 KB
/
UnsafeGeneratedAggregate.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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.trees._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.PlatformDependent
import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.unsafe.map.BytesToBytesMap
import org.apache.spark.unsafe.memory.MemoryAllocator
// TODO: finish cleaning up documentation instead of just copying it
/**
* TODO: copy of GeneratedAggregate that uses unsafe / offheap row implementations + hashtables.
*/
@DeveloperApi
case class UnsafeGeneratedAggregate(
partial: Boolean,
groupingExpressions: Seq[Expression],
aggregateExpressions: Seq[NamedExpression],
child: SparkPlan)
extends UnaryNode {
override def requiredChildDistribution: Seq[Distribution] =
if (partial) {
UnspecifiedDistribution :: Nil
} else {
if (groupingExpressions == Nil) {
AllTuples :: Nil
} else {
ClusteredDistribution(groupingExpressions) :: Nil
}
}
override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
override def execute(): RDD[Row] = {
val aggregatesToCompute = aggregateExpressions.flatMap { a =>
a.collect { case agg: AggregateExpression => agg}
}
// If you add any new function support, please add tests in org.apache.spark.sql.SQLQuerySuite
// (in test "aggregation with codegen").
val computeFunctions = aggregatesToCompute.map {
case c @ Count(expr) =>
// If we're evaluating UnscaledValue(x), we can do Count on x directly, since its
// UnscaledValue will be null if and only if x is null; helps with Average on decimals
val toCount = expr match {
case UnscaledValue(e) => e
case _ => expr
}
val currentCount = AttributeReference("currentCount", LongType, nullable = false)()
val initialValue = Literal(0L)
val updateFunction = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount)
val result = currentCount
AggregateEvaluation(currentCount :: Nil, initialValue :: Nil, updateFunction :: Nil, result)
case s @ Sum(expr) =>
val calcType =
expr.dataType match {
case DecimalType.Fixed(_, _) =>
DecimalType.Unlimited
case _ =>
expr.dataType
}
val currentSum = AttributeReference("currentSum", calcType, nullable = true)()
val initialValue = Literal.create(null, calcType)
// Coalesce avoids double calculation...
// but really, common sub expression elimination would be better....
val zero = Cast(Literal(0), calcType)
val updateFunction = Coalesce(
Add(
Coalesce(currentSum :: zero :: Nil),
Cast(expr, calcType)
) :: currentSum :: zero :: Nil)
val result =
expr.dataType match {
case DecimalType.Fixed(_, _) =>
Cast(currentSum, s.dataType)
case _ => currentSum
}
AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result)
case cs @ CombineSum(expr) =>
val calcType = expr.dataType
expr.dataType match {
case DecimalType.Fixed(_, _) =>
DecimalType.Unlimited
case _ =>
expr.dataType
}
val currentSum = AttributeReference("currentSum", calcType, nullable = true)()
val initialValue = Literal.create(null, calcType)
// Coalasce avoids double calculation...
// but really, common sub expression elimination would be better....
val zero = Cast(Literal(0), calcType)
// If we're evaluating UnscaledValue(x), we can do Count on x directly, since its
// UnscaledValue will be null if and only if x is null; helps with Average on decimals
val actualExpr = expr match {
case UnscaledValue(e) => e
case _ => expr
}
// partial sum result can be null only when no input rows present
val updateFunction = If(
IsNotNull(actualExpr),
Coalesce(
Add(
Coalesce(currentSum :: zero :: Nil),
Cast(expr, calcType)) :: currentSum :: zero :: Nil),
currentSum)
val result =
expr.dataType match {
case DecimalType.Fixed(_, _) =>
Cast(currentSum, cs.dataType)
case _ => currentSum
}
AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result)
case m @ Max(expr) =>
val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)()
val initialValue = Literal.create(null, expr.dataType)
val updateMax = MaxOf(currentMax, expr)
AggregateEvaluation(
currentMax :: Nil,
initialValue :: Nil,
updateMax :: Nil,
currentMax)
case m @ Min(expr) =>
val currentMin = AttributeReference("currentMin", expr.dataType, nullable = true)()
val initialValue = Literal.create(null, expr.dataType)
val updateMin = MinOf(currentMin, expr)
AggregateEvaluation(
currentMin :: Nil,
initialValue :: Nil,
updateMin :: Nil,
currentMin)
case CollectHashSet(Seq(expr)) =>
val set =
AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)()
val initialValue = NewSet(expr.dataType)
val addToSet = AddItemToSet(expr, set)
AggregateEvaluation(
set :: Nil,
initialValue :: Nil,
addToSet :: Nil,
set)
case CombineSetsAndCount(inputSet) =>
val elementType = inputSet.dataType.asInstanceOf[OpenHashSetUDT].elementType
val set =
AttributeReference("hashSet", new OpenHashSetUDT(elementType), nullable = false)()
val initialValue = NewSet(elementType)
val collectSets = CombineSets(set, inputSet)
AggregateEvaluation(
set :: Nil,
initialValue :: Nil,
collectSets :: Nil,
CountSet(set))
case o => sys.error(s"$o can't be codegened.")
}
val computationSchema: Seq[Attribute] = computeFunctions.flatMap(_.schema)
val resultMap: Map[TreeNodeRef, Expression] =
aggregatesToCompute.zip(computeFunctions).map {
case (agg, func) => new TreeNodeRef(agg) -> func.result
}.toMap
val namedGroups = groupingExpressions.zipWithIndex.map {
case (ne: NamedExpression, _) => (ne, ne)
case (e, i) => (e, Alias(e, s"GroupingExpr$i")())
}
val groupMap: Map[Expression, Attribute] =
namedGroups.map { case (k, v) => k -> v.toAttribute}.toMap
// The set of expressions that produce the final output given the aggregation buffer and the
// grouping expressions.
val resultExpressions = aggregateExpressions.map(_.transform {
case e: Expression if resultMap.contains(new TreeNodeRef(e)) => resultMap(new TreeNodeRef(e))
case e: Expression if groupMap.contains(e) => groupMap(e)
})
child.execute().mapPartitions { iter =>
// Builds a new custom class for holding the results of aggregation for a group.
val initialValues = computeFunctions.flatMap(_.initialValues)
val newAggregationBuffer = newProjection(initialValues, child.output)
log.info(s"Initial values: ${initialValues.mkString(",")}")
// A projection that computes the group given an input tuple.
val groupProjection = newProjection(groupingExpressions, child.output)
log.info(s"Grouping Projection: ${groupingExpressions.mkString(",")}")
// A projection that is used to update the aggregate values for a group given a new tuple.
// This projection should be targeted at the current values for the group and then applied
// to a joined row of the current values with the new input row.
val updateExpressions = computeFunctions.flatMap(_.update)
val updateSchema = computationSchema ++ child.output
val updateProjection = newMutableProjection(updateExpressions, updateSchema)()
log.info(s"Update Expressions: ${updateExpressions.mkString(",")}")
// A projection that produces the final result, given a computation.
val resultProjectionBuilder =
newMutableProjection(
resultExpressions,
(namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq)
log.info(s"Result Projection: ${resultExpressions.mkString(",")}")
val joinedRow = new JoinedRow3
if (groupingExpressions.isEmpty) {
// TODO: Codegening anything other than the updateProjection is probably over kill.
val buffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow]
var currentRow: Row = null
updateProjection.target(buffer)
while (iter.hasNext) {
currentRow = iter.next()
updateProjection(joinedRow(buffer, currentRow))
}
val resultProjection = resultProjectionBuilder()
Iterator(resultProjection(buffer))
} else {
// TODO: if we knew how many groups to expect, we could size this hashmap appropriately
val buffers = new BytesToBytesMap(MemoryAllocator.HEAP, 128)
// Set up the mutable "pointers" that we'll re-use when pointing to key and value rows
val keyPointer: UnsafeRow = new UnsafeRow()
val currentBuffer: UnsafeRow = new UnsafeRow()
// We're going to need to allocate a lot of empty aggregation buffers, so let's do it
// once and keep a copy of the serialized buffer and copy it into the hash map when we see
// new keys:
val emptyAggregationBuffer: Array[Long] = {
val javaBuffer: MutableRow = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow]
val fieldTypes = StructType.fromAttributes(computationSchema).map(_.dataType).toArray
val converter = new UnsafeRowConverter(fieldTypes)
val buffer = new Array[Long](converter.getSizeRequirement(javaBuffer))
converter.writeRow(javaBuffer, buffer, PlatformDependent.LONG_ARRAY_OFFSET)
buffer
}
val keyToUnsafeRowConverter: UnsafeRowConverter = {
new UnsafeRowConverter(groupingExpressions.map(_.dataType).toArray)
}
val aggregationBufferSchema = StructType.fromAttributes(computationSchema)
val keySchema: StructType = {
val fields = groupingExpressions.zipWithIndex.map { case (expr, idx) =>
StructField(idx.toString, expr.dataType, expr.nullable)
}
StructType(fields)
}
// Allocate some scratch space for holding the keys that we use to index into the hash map.
// 16 MB ought to be enough for anyone (TODO)
val unsafeRowBuffer: Array[Long] = new Array[Long](1024 * 16 / 8)
while (iter.hasNext) {
// Zero out the buffer that's used to hold the current row. This is necessary in order
// to ensure that rows hash properly, since garbage data from the previous row could
// otherwise end up as padding in this row.
ByteArrayMethods.zeroBytes(
unsafeRowBuffer, PlatformDependent.LONG_ARRAY_OFFSET, unsafeRowBuffer.length)
// Grab the next row from our input iterator and compute its group projection.
// In the long run, it might be nice to use Unsafe rows for this as well, but for now
// we'll just rely on the existing code paths to compute the projection.
val currentJavaRow = iter.next()
val currentGroup: Row = groupProjection(currentJavaRow)
// Convert the current group into an UnsafeRow so that we can use it as a key for our
// aggregation hash map
val groupProjectionSize = keyToUnsafeRowConverter.getSizeRequirement(currentGroup)
if (groupProjectionSize > unsafeRowBuffer.length) {
throw new IllegalStateException("Group projection does not fit into buffer")
}
val keyLengthInBytes: Int = keyToUnsafeRowConverter.writeRow(
currentGroup, unsafeRowBuffer, PlatformDependent.LONG_ARRAY_OFFSET).toInt // TODO
val loc: BytesToBytesMap#Location =
buffers.lookup(unsafeRowBuffer, PlatformDependent.LONG_ARRAY_OFFSET, keyLengthInBytes)
if (!loc.isDefined) {
// This is the first time that we've seen this key, so we'll copy the empty aggregation
// buffer row that we created earlier. TODO: this doesn't work very well for aggregates
// where the size of the aggregate buffer is different for different rows (even if the
// size of buffers don't grow once created, as is the case for things like grabbing the
// first row's value for a string-valued column (or the shortest string)).
loc.storeKeyAndValue(
unsafeRowBuffer,
PlatformDependent.LONG_ARRAY_OFFSET,
keyLengthInBytes,
emptyAggregationBuffer,
PlatformDependent.LONG_ARRAY_OFFSET,
emptyAggregationBuffer.length
)
// So that the pointers point to the value we just stored:
// TODO: reset this inside of the map so that this extra looup isn't necessary
buffers.lookup(unsafeRowBuffer, PlatformDependent.LONG_ARRAY_OFFSET, keyLengthInBytes)
}
// Reset our pointer to point to the buffer stored in the hash map
val address = loc.getValueAddress
currentBuffer.set(
address.getBaseObject,
address.getBaseOffset,
aggregationBufferSchema.length,
aggregationBufferSchema
)
// Target the projection at the current aggregation buffer and then project the updated
// values.
updateProjection.target(currentBuffer)(joinedRow(currentBuffer, currentJavaRow))
}
new Iterator[Row] {
private[this] val resultIterator = buffers.iterator()
private[this] val resultProjection = resultProjectionBuilder()
private[this] val key: UnsafeRow = new UnsafeRow()
private[this] val value: UnsafeRow = new UnsafeRow()
def hasNext: Boolean = resultIterator.hasNext
def next(): Row = {
val currentGroup: BytesToBytesMap#Location = resultIterator.next()
val keyAddress = currentGroup.getKeyAddress
key.set(
keyAddress.getBaseObject,
keyAddress.getBaseOffset,
groupingExpressions.length,
keySchema)
val valueAddress = currentGroup.getValueAddress
value.set(
valueAddress.getBaseObject,
valueAddress.getBaseOffset,
aggregationBufferSchema.length,
aggregationBufferSchema)
// TODO: once the iterator has been fully consumed, we need to free the map so that
// its off-heap memory is reclaimed. This may mean that we'll have to perform an extra
// defensive copy of the last row so that we can free that memory before returning
// to the caller.
resultProjection(joinedRow(key, value))
}
}
}
}
}
}