diff options
author | Marcelo Vanzin <vanzin@cloudera.com> | 2015-08-05 17:58:36 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2015-08-05 17:58:36 -0700 |
commit | 4399b7b0903d830313ab7e69731c11d587ae567c (patch) | |
tree | 156f583e44c7a45caac40a06b84e28bbe26a89d5 | |
parent | 8c320e45b5c9ffd7f0e35c1c7e6b5fc355377ea6 (diff) | |
download | spark-4399b7b0903d830313ab7e69731c11d587ae567c.tar.gz spark-4399b7b0903d830313ab7e69731c11d587ae567c.tar.bz2 spark-4399b7b0903d830313ab7e69731c11d587ae567c.zip |
[SPARK-9651] Fix UnsafeExternalSorterSuite.
First, it's probably a bad idea to call generated Scala methods
from Java. In this case, the method being called wasn't actually
"Utils.createTempDir()", but actually the method that returns the
first default argument to the actual createTempDir method, which
is just the location of java.io.tmpdir; meaning that all tests in
the class were using the same temp dir, and thus affecting each
other.
Second, spillingOccursInResponseToMemoryPressure was not writing
enough records to actually cause a spill.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #7970 from vanzin/SPARK-9651 and squashes the following commits:
74d357f [Marcelo Vanzin] Clean up temp dir on test tear down.
a64f36a [Marcelo Vanzin] [SPARK-9651] Fix UnsafeExternalSorterSuite.
-rw-r--r-- | core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java | 21 |
1 files changed, 13 insertions, 8 deletions
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 968185bde7..117745f9a9 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -101,7 +101,7 @@ public class UnsafeExternalSorterSuite { public void setUp() { MockitoAnnotations.initMocks(this); sparkConf = new SparkConf(); - tempDir = new File(Utils.createTempDir$default$1()); + tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); shuffleMemoryManager = new ShuffleMemoryManager(Long.MAX_VALUE); spillFilesCreated.clear(); taskContext = mock(TaskContext.class); @@ -143,13 +143,18 @@ public class UnsafeExternalSorterSuite { @After public void tearDown() { - long leakedUnsafeMemory = taskMemoryManager.cleanUpAllAllocatedMemory(); - if (shuffleMemoryManager != null) { - long leakedShuffleMemory = shuffleMemoryManager.getMemoryConsumptionForThisTask(); - shuffleMemoryManager = null; - assertEquals(0L, leakedShuffleMemory); + try { + long leakedUnsafeMemory = taskMemoryManager.cleanUpAllAllocatedMemory(); + if (shuffleMemoryManager != null) { + long leakedShuffleMemory = shuffleMemoryManager.getMemoryConsumptionForThisTask(); + shuffleMemoryManager = null; + assertEquals(0L, leakedShuffleMemory); + } + assertEquals(0, leakedUnsafeMemory); + } finally { + Utils.deleteRecursively(tempDir); + tempDir = null; } - assertEquals(0, leakedUnsafeMemory); } private void assertSpillFilesWereCleanedUp() { @@ -234,7 +239,7 @@ public class UnsafeExternalSorterSuite { public void spillingOccursInResponseToMemoryPressure() throws Exception { shuffleMemoryManager = new ShuffleMemoryManager(pageSizeBytes * 2); final UnsafeExternalSorter sorter = newSorter(); - final int numRecords = 100000; + final int numRecords = (int) pageSizeBytes / 4; for (int i = 0; i <= numRecords; i++) { insertNumber(sorter, numRecords - i); } |