@@ -19,12 +19,12 @@ package org.apache.spark.util.collection
19
19
20
20
import scala .collection .mutable .ArrayBuffer
21
21
22
- import org .scalatest .FunSuite
22
+ import org .scalatest .{ PrivateMethodTester , FunSuite }
23
23
24
24
import org .apache .spark ._
25
25
import org .apache .spark .SparkContext ._
26
26
27
- class ExternalSorterSuite extends FunSuite with LocalSparkContext {
27
+ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester {
28
28
private def createSparkConf (loadDefaults : Boolean ): SparkConf = {
29
29
val conf = new SparkConf (loadDefaults)
30
30
// Make the Java serializer write a reset instruction (TC_RESET) after each object to test
@@ -36,6 +36,16 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
36
36
conf
37
37
}
38
38
39
+ private def assertBypassedMergeSort (sorter : ExternalSorter [_, _, _]): Unit = {
40
+ val bypassMergeSort = PrivateMethod [Boolean ](' bypassMergeSort )
41
+ assert(sorter.invokePrivate(bypassMergeSort()), " sorter did not bypass merge-sort" )
42
+ }
43
+
44
+ private def assertDidNotBypassMergeSort (sorter : ExternalSorter [_, _, _]): Unit = {
45
+ val bypassMergeSort = PrivateMethod [Boolean ](' bypassMergeSort )
46
+ assert(! sorter.invokePrivate(bypassMergeSort()), " sorter bypassed merge-sort" )
47
+ }
48
+
39
49
test(" empty data stream" ) {
40
50
val conf = new SparkConf (false )
41
51
conf.set(" spark.shuffle.memoryFraction" , " 0.001" )
@@ -123,7 +133,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
123
133
124
134
val sorter = new ExternalSorter [Int , Int , Int ](
125
135
None , Some (new HashPartitioner (7 )), Some (ord), None )
126
- assert( ! sorter.bypassMergeSort, " sorter bypassed merge-sort " )
136
+ assertDidNotBypassMergeSort( sorter)
127
137
sorter.insertAll(elements)
128
138
assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0 ) // Make sure it spilled
129
139
val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList))
@@ -147,7 +157,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
147
157
148
158
val sorter = new ExternalSorter [Int , Int , Int ](
149
159
None , Some (new HashPartitioner (7 )), None , None )
150
- assert (sorter.bypassMergeSort, " sorter did not bypass merge-sort " )
160
+ assertBypassedMergeSort (sorter)
151
161
sorter.insertAll(elements)
152
162
assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0 ) // Make sure it spilled
153
163
val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList))
@@ -314,15 +324,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
314
324
315
325
val sorter = new ExternalSorter [Int , Int , Int ](
316
326
None , Some (new HashPartitioner (3 )), Some (ord), None )
317
- assert( ! sorter.bypassMergeSort, " sorter bypassed merge-sort " )
327
+ assertDidNotBypassMergeSort( sorter)
318
328
sorter.insertAll((0 until 100000 ).iterator.map(i => (i, i)))
319
329
assert(diskBlockManager.getAllFiles().length > 0 )
320
330
sorter.stop()
321
331
assert(diskBlockManager.getAllBlocks().length === 0 )
322
332
323
333
val sorter2 = new ExternalSorter [Int , Int , Int ](
324
334
None , Some (new HashPartitioner (3 )), Some (ord), None )
325
- assert( ! sorter2.bypassMergeSort, " sorter bypassed merge-sort " )
335
+ assertDidNotBypassMergeSort( sorter2)
326
336
sorter2.insertAll((0 until 100000 ).iterator.map(i => (i, i)))
327
337
assert(diskBlockManager.getAllFiles().length > 0 )
328
338
assert(sorter2.iterator.toSet === (0 until 100000 ).map(i => (i, i)).toSet)
@@ -338,14 +348,14 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
338
348
val diskBlockManager = SparkEnv .get.blockManager.diskBlockManager
339
349
340
350
val sorter = new ExternalSorter [Int , Int , Int ](None , Some (new HashPartitioner (3 )), None , None )
341
- assert (sorter.bypassMergeSort, " sorter did not bypass merge-sort " )
351
+ assertBypassedMergeSort (sorter)
342
352
sorter.insertAll((0 until 100000 ).iterator.map(i => (i, i)))
343
353
assert(diskBlockManager.getAllFiles().length > 0 )
344
354
sorter.stop()
345
355
assert(diskBlockManager.getAllBlocks().length === 0 )
346
356
347
357
val sorter2 = new ExternalSorter [Int , Int , Int ](None , Some (new HashPartitioner (3 )), None , None )
348
- assert (sorter2.bypassMergeSort, " sorter did not bypass merge-sort " )
358
+ assertBypassedMergeSort (sorter2)
349
359
sorter2.insertAll((0 until 100000 ).iterator.map(i => (i, i)))
350
360
assert(diskBlockManager.getAllFiles().length > 0 )
351
361
assert(sorter2.iterator.toSet === (0 until 100000 ).map(i => (i, i)).toSet)
@@ -364,7 +374,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
364
374
365
375
val sorter = new ExternalSorter [Int , Int , Int ](
366
376
None , Some (new HashPartitioner (3 )), Some (ord), None )
367
- assert( ! sorter.bypassMergeSort, " sorter bypassed merge-sort " )
377
+ assertDidNotBypassMergeSort( sorter)
368
378
intercept[SparkException ] {
369
379
sorter.insertAll((0 until 100000 ).iterator.map(i => {
370
380
if (i == 99990 ) {
@@ -386,7 +396,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
386
396
val diskBlockManager = SparkEnv .get.blockManager.diskBlockManager
387
397
388
398
val sorter = new ExternalSorter [Int , Int , Int ](None , Some (new HashPartitioner (3 )), None , None )
389
- assert (sorter.bypassMergeSort, " sorter did not bypass merge-sort " )
399
+ assertBypassedMergeSort (sorter)
390
400
intercept[SparkException ] {
391
401
sorter.insertAll((0 until 100000 ).iterator.map(i => {
392
402
if (i == 99990 ) {
@@ -681,20 +691,20 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext {
681
691
682
692
val sorter1 = new ExternalSorter [Int , Int , Int ](
683
693
None , Some (new HashPartitioner (FEW_PARTITIONS )), None , None )
684
- assert (sorter1.bypassMergeSort, " sorter did not bypass merge-sort " )
694
+ assertBypassedMergeSort (sorter1)
685
695
686
696
val sorter2 = new ExternalSorter [Int , Int , Int ](
687
697
None , Some (new HashPartitioner (MANY_PARTITIONS )), None , None )
688
- assert( ! sorter2.bypassMergeSort, " sorter bypassed merge-sort " )
698
+ assertDidNotBypassMergeSort( sorter2)
689
699
690
700
// Sorters with an ordering or aggregator: should not bypass even if they have few partitions
691
701
692
702
val sorter3 = new ExternalSorter [Int , Int , Int ](
693
703
None , Some (new HashPartitioner (FEW_PARTITIONS )), Some (ord), None )
694
- assert( ! sorter3.bypassMergeSort, " sorter bypassed merge-sort " )
704
+ assertDidNotBypassMergeSort( sorter3)
695
705
696
706
val sorter4 = new ExternalSorter [Int , Int , Int ](
697
707
Some (agg), Some (new HashPartitioner (FEW_PARTITIONS )), None , None )
698
- assert( ! sorter4.bypassMergeSort, " sorter bypassed merge-sort " )
708
+ assertDidNotBypassMergeSort( sorter4)
699
709
}
700
710
}
0 commit comments