forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 3
Fix tests to not ignore ordering and also assert all data is present #18
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.streaming.util | |
| import java.io._ | ||
| import java.nio.ByteBuffer | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.fs.{FileStatus, Path} | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.concurrent.duration._ | ||
|
|
@@ -41,14 +41,14 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| val hadoopConf = new Configuration() | ||
| val dfsDir = Files.createTempDir() | ||
| val TEST_BUILD_DATA_KEY: String = "test.build.data" | ||
| val oldTestBuildDataProp = System.getProperty(TEST_BUILD_DATA_KEY) | ||
| val oldTestBuildDataProp = Option(System.getProperty(TEST_BUILD_DATA_KEY)) | ||
| System.setProperty(TEST_BUILD_DATA_KEY, dfsDir.toString) | ||
| val cluster = new MiniDFSCluster(new Configuration, 2, true, null) | ||
| val nnPort = cluster.getNameNode.getNameNodeAddress.getPort | ||
| val hdfsUrl = s"hdfs://localhost:$nnPort/${getRandomString()}/" | ||
| val hdfsUrl = s"hdfs://localhost:$nnPort/${getRandomString()}/" | ||
| var pathForTest: String = null | ||
|
|
||
| override def beforeAll() { | ||
| System.setProperty(TEST_BUILD_DATA_KEY, dfsDir.toString) | ||
| cluster.waitActive() | ||
| } | ||
|
|
||
|
|
@@ -59,7 +59,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| override def afterAll() { | ||
| cluster.shutdown() | ||
| FileUtils.deleteDirectory(dfsDir) | ||
| System.setProperty(TEST_BUILD_DATA_KEY, oldTestBuildDataProp) | ||
| oldTestBuildDataProp.foreach(System.setProperty(TEST_BUILD_DATA_KEY, _)) | ||
| } | ||
|
|
||
| test("WriteAheadLogWriter - writing data") { | ||
|
|
@@ -71,8 +71,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| assert(writtenData.toArray === dataToWrite.toArray) | ||
| } | ||
|
|
||
| test("WriteAheadLogWriter - syncing of data by writing and reading immediately using " + | ||
| "Minicluster") { | ||
| test("WriteAheadLogWriter - syncing of data by writing and reading immediately") { | ||
| val dataToWrite = generateRandomData() | ||
| val writer = new WriteAheadLogWriter(pathForTest, hadoopConf) | ||
| dataToWrite.foreach { data => | ||
|
|
@@ -98,7 +97,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| reader.close() | ||
| } | ||
|
|
||
| test("WriteAheadLogReader - sequentially reading data written with writer using Minicluster") { | ||
| test("WriteAheadLogReader - sequentially reading data written with writer") { | ||
| // Write data manually for testing the sequential reader | ||
| val dataToWrite = generateRandomData() | ||
| writeDataUsingWriter(pathForTest, dataToWrite) | ||
|
|
@@ -124,8 +123,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| reader.close() | ||
| } | ||
|
|
||
| test("WriteAheadLogRandomReader - reading data using random reader written with writer using " + | ||
| "Minicluster") { | ||
| test("WriteAheadLogRandomReader - reading data using random reader written with writer") { | ||
| // Write data using writer for testing the random reader | ||
| val data = generateRandomData() | ||
| val segments = writeDataUsingWriter(pathForTest, data) | ||
|
|
@@ -141,24 +139,23 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
|
|
||
| test("WriteAheadLogManager - write rotating logs") { | ||
| // Write data using manager | ||
| val dataToWrite = generateRandomData(10) | ||
| val dataToWrite = generateRandomData() | ||
| val dir = pathForTest | ||
| writeDataUsingManager(dir, dataToWrite) | ||
|
|
||
| // Read data manually to verify the written data | ||
| val logFiles = getLogFilesInDirectory(dir) | ||
| assert(logFiles.size > 1) | ||
| val writtenData = logFiles.flatMap { file => readDataManually(file) } | ||
| assert(writtenData.toSet === dataToWrite.toSet) | ||
| val writtenData = logFiles.flatMap { file => readDataManually(file)} | ||
| assert(writtenData.toList === dataToWrite.toList) | ||
| } | ||
|
|
||
| // This one is failing right now -- commenting out for now. | ||
| test("WriteAheadLogManager - read rotating logs") { | ||
| // Write data manually for testing reading through manager | ||
| val dir = pathForTest | ||
| val writtenData = (1 to 10).map { i => | ||
| val data = generateRandomData(10) | ||
| val file = dir + "/log-" + i | ||
| val data = generateRandomData() | ||
| val file = dir + s"/log-$i-$i" | ||
| writeDataManually(data, file) | ||
| data | ||
| }.flatten | ||
|
|
@@ -169,12 +166,12 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
|
|
||
| // Read data using manager and verify | ||
| val readData = readDataUsingManager(dir) | ||
| // assert(readData.toList === writtenData.toList) | ||
| assert(readData.toList === writtenData.toList) | ||
| } | ||
|
|
||
| test("WriteAheadLogManager - recover past logs when creating new manager") { | ||
| // Write data with manager, recover with new manager and verify | ||
| val dataToWrite = generateRandomData(100) | ||
| val dataToWrite = generateRandomData() | ||
| val dir = pathForTest | ||
| writeDataUsingManager(dir, dataToWrite) | ||
| val logFiles = getLogFilesInDirectory(dir) | ||
|
|
@@ -186,7 +183,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| test("WriteAheadLogManager - cleanup old logs") { | ||
| // Write data with manager, recover with new manager and verify | ||
| val dir = pathForTest | ||
| val dataToWrite = generateRandomData(100) | ||
| val dataToWrite = generateRandomData() | ||
| val fakeClock = new ManualClock | ||
| val manager = new WriteAheadLogManager(dir, hadoopConf, | ||
| rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = fakeClock) | ||
|
|
@@ -201,7 +198,6 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte | |
| assert(getLogFilesInDirectory(dir).size < logFiles.size) | ||
| } | ||
| } | ||
|
|
||
| // TODO (Hari, TD): Test different failure conditions of writers and readers. | ||
| // - Failure while reading incomplete/corrupt file | ||
| } | ||
|
|
@@ -243,8 +239,10 @@ object WriteAheadLogSuite { | |
|
|
||
| def writeDataUsingManager(logDirectory: String, data: Seq[String]) { | ||
| val fakeClock = new ManualClock | ||
| fakeClock.setTime(1000000) | ||
| val manager = new WriteAheadLogManager(logDirectory, hadoopConf, | ||
| rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = fakeClock) | ||
| // Ensure that 500 does not get sorted after 2000, so put a high base value. | ||
| data.foreach { item => | ||
| fakeClock.addToTime(500) | ||
| manager.writeToLog(item) | ||
|
|
@@ -271,7 +269,8 @@ object WriteAheadLogSuite { | |
| val reader = HdfsUtils.getInputStream(file, hadoopConf) | ||
| val buffer = new ArrayBuffer[String] | ||
| try { | ||
| while (true) { // Read till EOF is thrown | ||
| while (true) { | ||
| // Read till EOF is thrown | ||
| val length = reader.readInt() | ||
| val bytes = new Array[Byte](length) | ||
| reader.read(bytes) | ||
|
|
@@ -293,8 +292,10 @@ object WriteAheadLogSuite { | |
| data | ||
| } | ||
|
|
||
| def generateRandomData(numItems: Int = 50, itemSize: Int = 50): Seq[String] = { | ||
| (1 to numItems).map { _.toString } | ||
| def generateRandomData(): Seq[String] = { | ||
| (1 to 50).map { | ||
| _.toString | ||
|
Owner
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. itemSize is not used. Remove.
Owner
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We probably dont need numItems either. Only used in one place, which can be removed to default. |
||
| } | ||
| } | ||
|
|
||
| def getLogFilesInDirectory(directory: String): Seq[String] = { | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch! But this is a baaaaaaaad hidden non-determinism. Not good. I will fix it in the actually code logic, to sort semantically by parsed start time, and not alphabetically.