@@ -19,16 +19,15 @@ package org.apache.spark.sql.execution
19
19
20
20
import java .util .{HashMap => JavaHashMap }
21
21
22
- import scala .collection .mutable .{ArrayBuffer , BitSet }
23
22
import scala .concurrent .ExecutionContext .Implicits .global
24
23
import scala .concurrent ._
25
24
import scala .concurrent .duration ._
26
25
27
26
import org .apache .spark .annotation .DeveloperApi
28
- import org .apache .spark .sql .SQLContext
29
27
import org .apache .spark .sql .catalyst .expressions ._
30
28
import org .apache .spark .sql .catalyst .plans ._
31
29
import org .apache .spark .sql .catalyst .plans .physical ._
30
+ import org .apache .spark .util .collection .CompactBuffer
32
31
33
32
@ DeveloperApi
34
33
sealed abstract class BuildSide
@@ -67,7 +66,7 @@ trait HashJoin {
67
66
def joinIterators (buildIter : Iterator [Row ], streamIter : Iterator [Row ]): Iterator [Row ] = {
68
67
// TODO: Use Spark's HashMap implementation.
69
68
70
- val hashTable = new java.util.HashMap [Row , ArrayBuffer [Row ]]()
69
+ val hashTable = new java.util.HashMap [Row , CompactBuffer [Row ]]()
71
70
var currentRow : Row = null
72
71
73
72
// Create a mapping of buildKeys -> rows
@@ -77,7 +76,7 @@ trait HashJoin {
77
76
if (! rowKey.anyNull) {
78
77
val existingMatchList = hashTable.get(rowKey)
79
78
val matchList = if (existingMatchList == null ) {
80
- val newMatchList = new ArrayBuffer [Row ]()
79
+ val newMatchList = new CompactBuffer [Row ]()
81
80
hashTable.put(rowKey, newMatchList)
82
81
newMatchList
83
82
} else {
@@ -89,7 +88,7 @@ trait HashJoin {
89
88
90
89
new Iterator [Row ] {
91
90
private [this ] var currentStreamedRow : Row = _
92
- private [this ] var currentHashMatches : ArrayBuffer [Row ] = _
91
+ private [this ] var currentHashMatches : CompactBuffer [Row ] = _
93
92
private [this ] var currentMatchPosition : Int = - 1
94
93
95
94
// Mutable per row objects.
@@ -140,7 +139,7 @@ trait HashJoin {
140
139
141
140
/**
142
141
* :: DeveloperApi ::
143
- * Performs a hash based outer join for two child relations by shuffling the data using
142
+ * Performs a hash based outer join for two child relations by shuffling the data using
144
143
* the join keys. This operator requires loading the associated partition in both side into memory.
145
144
*/
146
145
@ DeveloperApi
@@ -179,26 +178,26 @@ case class HashOuterJoin(
179
178
@ transient private [this ] lazy val EMPTY_LIST = Seq .empty[Row ]
180
179
181
180
// TODO we need to rewrite all of the iterators with our own implementation instead of the Scala
182
- // iterator for performance purpose.
181
+ // iterator for performance purpose.
183
182
184
183
private [this ] def leftOuterIterator (
185
184
key : Row , leftIter : Iterable [Row ], rightIter : Iterable [Row ]): Iterator [Row ] = {
186
185
val joinedRow = new JoinedRow ()
187
186
val rightNullRow = new GenericRow (right.output.length)
188
- val boundCondition =
187
+ val boundCondition =
189
188
condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row : Row ) => true )
190
189
191
- leftIter.iterator.flatMap { l =>
190
+ leftIter.iterator.flatMap { l =>
192
191
joinedRow.withLeft(l)
193
192
var matched = false
194
- (if (! key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) =>
193
+ (if (! key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) =>
195
194
matched = true
196
195
joinedRow.copy
197
196
} else {
198
197
Nil
199
198
}) ++ DUMMY_LIST .filter(_ => ! matched).map( _ => {
200
199
// DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row,
201
- // as we don't know whether we need to append it until finish iterating all of the
200
+ // as we don't know whether we need to append it until finish iterating all of the
202
201
// records in right side.
203
202
// If we didn't get any proper row, then append a single row with empty right
204
203
joinedRow.withRight(rightNullRow).copy
@@ -210,20 +209,20 @@ case class HashOuterJoin(
210
209
key : Row , leftIter : Iterable [Row ], rightIter : Iterable [Row ]): Iterator [Row ] = {
211
210
val joinedRow = new JoinedRow ()
212
211
val leftNullRow = new GenericRow (left.output.length)
213
- val boundCondition =
212
+ val boundCondition =
214
213
condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row : Row ) => true )
215
214
216
- rightIter.iterator.flatMap { r =>
215
+ rightIter.iterator.flatMap { r =>
217
216
joinedRow.withRight(r)
218
217
var matched = false
219
- (if (! key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) =>
218
+ (if (! key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) =>
220
219
matched = true
221
220
joinedRow.copy
222
221
} else {
223
222
Nil
224
223
}) ++ DUMMY_LIST .filter(_ => ! matched).map( _ => {
225
224
// DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row,
226
- // as we don't know whether we need to append it until finish iterating all of the
225
+ // as we don't know whether we need to append it until finish iterating all of the
227
226
// records in left side.
228
227
// If we didn't get any proper row, then append a single row with empty left.
229
228
joinedRow.withLeft(leftNullRow).copy
@@ -236,7 +235,7 @@ case class HashOuterJoin(
236
235
val joinedRow = new JoinedRow ()
237
236
val leftNullRow = new GenericRow (left.output.length)
238
237
val rightNullRow = new GenericRow (right.output.length)
239
- val boundCondition =
238
+ val boundCondition =
240
239
condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row : Row ) => true )
241
240
242
241
if (! key.anyNull) {
@@ -246,8 +245,8 @@ case class HashOuterJoin(
246
245
leftIter.iterator.flatMap[Row ] { l =>
247
246
joinedRow.withLeft(l)
248
247
var matched = false
249
- rightIter.zipWithIndex.collect {
250
- // 1. For those matched (satisfy the join condition) records with both sides filled,
248
+ rightIter.zipWithIndex.collect {
249
+ // 1. For those matched (satisfy the join condition) records with both sides filled,
251
250
// append them directly
252
251
253
252
case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> {
@@ -260,16 +259,16 @@ case class HashOuterJoin(
260
259
// 2. For those unmatched records in left, append additional records with empty right.
261
260
262
261
// DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row,
263
- // as we don't know whether we need to append it until finish iterating all
262
+ // as we don't know whether we need to append it until finish iterating all
264
263
// of the records in right side.
265
264
// If we didn't get any proper row, then append a single row with empty right.
266
265
joinedRow.withRight(rightNullRow).copy
267
266
})
268
267
} ++ rightIter.zipWithIndex.collect {
269
268
// 3. For those unmatched records in right, append additional records with empty left.
270
269
271
- // Re-visiting the records in right, and append additional row with empty left, if its not
272
- // in the matched set.
270
+ // Re-visiting the records in right, and append additional row with empty left, if its not
271
+ // in the matched set.
273
272
case (r, idx) if (! rightMatchedSet.contains(idx)) => {
274
273
joinedRow(leftNullRow, r).copy
275
274
}
@@ -284,15 +283,15 @@ case class HashOuterJoin(
284
283
}
285
284
286
285
private [this ] def buildHashTable (
287
- iter : Iterator [Row ], keyGenerator : Projection ): JavaHashMap [Row , ArrayBuffer [Row ]] = {
288
- val hashTable = new JavaHashMap [Row , ArrayBuffer [Row ]]()
286
+ iter : Iterator [Row ], keyGenerator : Projection ): JavaHashMap [Row , CompactBuffer [Row ]] = {
287
+ val hashTable = new JavaHashMap [Row , CompactBuffer [Row ]]()
289
288
while (iter.hasNext) {
290
289
val currentRow = iter.next()
291
290
val rowKey = keyGenerator(currentRow)
292
291
293
292
var existingMatchList = hashTable.get(rowKey)
294
293
if (existingMatchList == null ) {
295
- existingMatchList = new ArrayBuffer [Row ]()
294
+ existingMatchList = new CompactBuffer [Row ]()
296
295
hashTable.put(rowKey, existingMatchList)
297
296
}
298
297
@@ -311,20 +310,20 @@ case class HashOuterJoin(
311
310
val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output))
312
311
313
312
import scala .collection .JavaConversions ._
314
- val boundCondition =
313
+ val boundCondition =
315
314
condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row : Row ) => true )
316
315
joinType match {
317
316
case LeftOuter => leftHashTable.keysIterator.flatMap { key =>
318
- leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST ),
317
+ leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST ),
319
318
rightHashTable.getOrElse(key, EMPTY_LIST ))
320
319
}
321
320
case RightOuter => rightHashTable.keysIterator.flatMap { key =>
322
- rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST ),
321
+ rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST ),
323
322
rightHashTable.getOrElse(key, EMPTY_LIST ))
324
323
}
325
324
case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key =>
326
- fullOuterIterator(key,
327
- leftHashTable.getOrElse(key, EMPTY_LIST ),
325
+ fullOuterIterator(key,
326
+ leftHashTable.getOrElse(key, EMPTY_LIST ),
328
327
rightHashTable.getOrElse(key, EMPTY_LIST ))
329
328
}
330
329
case x => throw new Exception (s " HashOuterJoin should not take $x as the JoinType " )
@@ -550,7 +549,7 @@ case class BroadcastNestedLoopJoin(
550
549
551
550
/** All rows that either match both-way, or rows from streamed joined with nulls. */
552
551
val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter =>
553
- val matchedRows = new ArrayBuffer [Row ]
552
+ val matchedRows = new CompactBuffer [Row ]
554
553
// TODO: Use Spark's BitSet.
555
554
val includedBroadcastTuples =
556
555
new scala.collection.mutable.BitSet (broadcastedRelation.value.size)
@@ -602,20 +601,20 @@ case class BroadcastNestedLoopJoin(
602
601
val rightNulls = new GenericMutableRow (right.output.size)
603
602
/** Rows from broadcasted joined with nulls. */
604
603
val broadcastRowsWithNulls : Seq [Row ] = {
605
- val arrBuf : collection.mutable. ArrayBuffer [Row ] = collection.mutable. ArrayBuffer ()
604
+ val buf : CompactBuffer [Row ] = new CompactBuffer ()
606
605
var i = 0
607
606
val rel = broadcastedRelation.value
608
607
while (i < rel.length) {
609
608
if (! allIncludedBroadcastTuples.contains(i)) {
610
609
(joinType, buildSide) match {
611
- case (RightOuter | FullOuter , BuildRight ) => arrBuf += new JoinedRow (leftNulls, rel(i))
612
- case (LeftOuter | FullOuter , BuildLeft ) => arrBuf += new JoinedRow (rel(i), rightNulls)
610
+ case (RightOuter | FullOuter , BuildRight ) => buf += new JoinedRow (leftNulls, rel(i))
611
+ case (LeftOuter | FullOuter , BuildLeft ) => buf += new JoinedRow (rel(i), rightNulls)
613
612
case _ =>
614
613
}
615
614
}
616
615
i += 1
617
616
}
618
- arrBuf .toSeq
617
+ buf .toSeq
619
618
}
620
619
621
620
// TODO: Breaks lineage.
0 commit comments