Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
assert(fakeClassVersion === "1")
val fakeClass2 = classLoader.loadClass("FakeClass2").newInstance()
assert(fakeClass.getClass === fakeClass2.getClass)
classLoader.close()
parentLoader.close()
}

test("parent first") {
Expand All @@ -61,6 +63,8 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
assert(fakeClassVersion === "2")
val fakeClass2 = classLoader.loadClass("FakeClass1").newInstance()
assert(fakeClass.getClass === fakeClass2.getClass)
classLoader.close()
parentLoader.close()
}

test("child first can fall back") {
Expand All @@ -69,6 +73,8 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
val fakeClass = classLoader.loadClass("FakeClass3").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
classLoader.close()
parentLoader.close()
}

test("child first can fail") {
Expand All @@ -77,20 +83,26 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {
intercept[java.lang.ClassNotFoundException] {
classLoader.loadClass("FakeClassDoesNotExist").newInstance()
}
classLoader.close()
parentLoader.close()
}

test("default JDK classloader get resources") {
val parentLoader = new URLClassLoader(fileUrlsParent, null)
val classLoader = new URLClassLoader(fileUrlsChild, parentLoader)
assert(classLoader.getResources("resource1").asScala.size === 2)
assert(classLoader.getResources("resource2").asScala.size === 1)
classLoader.close()
parentLoader.close()
}

test("parent first get resources") {
val parentLoader = new URLClassLoader(fileUrlsParent, null)
val classLoader = new MutableURLClassLoader(fileUrlsChild, parentLoader)
assert(classLoader.getResources("resource1").asScala.size === 2)
assert(classLoader.getResources("resource2").asScala.size === 1)
classLoader.close()
parentLoader.close()
}

test("child first get resources") {
Expand All @@ -103,6 +115,8 @@ class MutableURLClassLoaderSuite extends SparkFunSuite with Matchers {

res1.map(scala.io.Source.fromURL(_).mkString) should contain inOrderOnly
("resource1Contents-child", "resource1Contents-parent")
classLoader.close()
parentLoader.close()
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,8 @@ object LocalFileLR {

showWarning()

val lines = scala.io.Source.fromFile(args(0)).getLines().toArray
val fileSrc = scala.io.Source.fromFile(args(0))
val lines = fileSrc.getLines().toArray
val points = lines.map(parsePoint _)
val ITERATIONS = args(1).toInt

Expand All @@ -69,6 +70,7 @@ object LocalFileLR {
w -= gradient
}

fileSrc.close()
println("Final w: " + w)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -272,6 +272,7 @@ class DirectKafkaStreamSuite
collectedData.contains("b")
}
assert(!collectedData.contains("a"))
ssc.stop()
}


Expand Down Expand Up @@ -324,6 +325,7 @@ class DirectKafkaStreamSuite
collectedData.contains("b")
}
assert(!collectedData.contains("a"))
ssc.stop()
}

// Test to verify the offset ranges can be recovered from the checkpoints
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,7 @@ class DirectKafkaStreamSuite
collectedData.contains("b")
}
assert(!collectedData.contains("a"))
ssc.stop()
}


Expand Down Expand Up @@ -230,6 +231,7 @@ class DirectKafkaStreamSuite
collectedData.contains("b")
}
assert(!collectedData.contains("a"))
ssc.stop()
}

// Test to verify the offset ranges can be recovered from the checkpoints
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,5 +80,6 @@ class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter
eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
assert(result.synchronized { sent === result })
}
ssc.stop()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -609,7 +609,12 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
test("SPARK-11043 check operation log root directory") {
val expectedLine =
"Operation log root directory is created: " + operationLogPath.getAbsoluteFile
assert(Source.fromFile(logPath).getLines().exists(_.contains(expectedLine)))
val bufferSrc = Source.fromFile(logPath)
Utils.tryWithSafeFinally {
assert(bufferSrc.getLines().exists(_.contains(expectedLine)))
} {
bufferSrc.close()
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -815,6 +815,7 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester
val ois = new ObjectInputStreamWithLoader(
new ByteArrayInputStream(bos.toByteArray), loader)
assert(ois.readObject().asInstanceOf[Class[_]].getName == "[LtestClz;")
ois.close()
}

test("SPARK-11267: the race condition of two checkpoints in a batch") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ class JobGeneratorSuite extends TestSuiteBase {
assert(getBlocksOfBatch(longBatchTime).nonEmpty, "blocks of incomplete batch already deleted")
assert(batchCounter.getNumCompletedBatches < longBatchNumber)
waitLatch.countDown()
ssc.stop()
}
}
}
Expand Down