Skip to content

Commit 5ff90ee

Browse files
Fix tests to not ignore ordering and also assert all data is present
1 parent 7e40e56 commit 5ff90ee

File tree

1 file changed

+21
-19
lines changed

1 file changed

+21
-19
lines changed

streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala

Lines changed: 21 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.streaming.util
1919
import java.io._
2020
import java.nio.ByteBuffer
2121

22-
import org.apache.hadoop.fs.Path
22+
import org.apache.hadoop.fs.{FileStatus, Path}
2323

2424
import scala.collection.mutable.ArrayBuffer
2525
import scala.concurrent.duration._
@@ -41,14 +41,14 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
4141
val hadoopConf = new Configuration()
4242
val dfsDir = Files.createTempDir()
4343
val TEST_BUILD_DATA_KEY: String = "test.build.data"
44-
val oldTestBuildDataProp = System.getProperty(TEST_BUILD_DATA_KEY)
44+
val oldTestBuildDataProp = Option(System.getProperty(TEST_BUILD_DATA_KEY))
45+
System.setProperty(TEST_BUILD_DATA_KEY, dfsDir.toString)
4546
val cluster = new MiniDFSCluster(new Configuration, 2, true, null)
4647
val nnPort = cluster.getNameNode.getNameNodeAddress.getPort
47-
val hdfsUrl = s"hdfs://localhost:$nnPort/${getRandomString()}/"
48+
val hdfsUrl = s"hdfs://localhost:$nnPort/${getRandomString()}/"
4849
var pathForTest: String = null
4950

5051
override def beforeAll() {
51-
System.setProperty(TEST_BUILD_DATA_KEY, dfsDir.toString)
5252
cluster.waitActive()
5353
}
5454

@@ -59,7 +59,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
5959
override def afterAll() {
6060
cluster.shutdown()
6161
FileUtils.deleteDirectory(dfsDir)
62-
System.setProperty(TEST_BUILD_DATA_KEY, oldTestBuildDataProp)
62+
oldTestBuildDataProp.foreach(System.setProperty(TEST_BUILD_DATA_KEY, _))
6363
}
6464

6565
test("WriteAheadLogWriter - writing data") {
@@ -71,8 +71,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
7171
assert(writtenData.toArray === dataToWrite.toArray)
7272
}
7373

74-
test("WriteAheadLogWriter - syncing of data by writing and reading immediately using " +
75-
"Minicluster") {
74+
test("WriteAheadLogWriter - syncing of data by writing and reading immediately") {
7675
val dataToWrite = generateRandomData()
7776
val writer = new WriteAheadLogWriter(pathForTest, hadoopConf)
7877
dataToWrite.foreach { data =>
@@ -98,7 +97,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
9897
reader.close()
9998
}
10099

101-
test("WriteAheadLogReader - sequentially reading data written with writer using Minicluster") {
100+
test("WriteAheadLogReader - sequentially reading data written with writer") {
102101
// Write data manually for testing the sequential reader
103102
val dataToWrite = generateRandomData()
104103
writeDataUsingWriter(pathForTest, dataToWrite)
@@ -124,8 +123,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
124123
reader.close()
125124
}
126125

127-
test("WriteAheadLogRandomReader - reading data using random reader written with writer using " +
128-
"Minicluster") {
126+
test("WriteAheadLogRandomReader - reading data using random reader written with writer") {
129127
// Write data using writer for testing the random reader
130128
val data = generateRandomData()
131129
val segments = writeDataUsingWriter(pathForTest, data)
@@ -148,17 +146,16 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
148146
// Read data manually to verify the written data
149147
val logFiles = getLogFilesInDirectory(dir)
150148
assert(logFiles.size > 1)
151-
val writtenData = logFiles.flatMap { file => readDataManually(file) }
152-
assert(writtenData.toSet === dataToWrite.toSet)
149+
val writtenData = logFiles.flatMap { file => readDataManually(file)}
150+
assert(writtenData.toList === dataToWrite.toList)
153151
}
154152

155-
// This one is failing right now -- commenting out for now.
156153
test("WriteAheadLogManager - read rotating logs") {
157154
// Write data manually for testing reading through manager
158155
val dir = pathForTest
159156
val writtenData = (1 to 10).map { i =>
160157
val data = generateRandomData(10)
161-
val file = dir + "/log-" + i
158+
val file = dir + s"/log-$i-$i"
162159
writeDataManually(data, file)
163160
data
164161
}.flatten
@@ -169,7 +166,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
169166

170167
// Read data using manager and verify
171168
val readData = readDataUsingManager(dir)
172-
// assert(readData.toList === writtenData.toList)
169+
assert(readData.toList === writtenData.toList)
173170
}
174171

175172
test("WriteAheadLogManager - recover past logs when creating new manager") {
@@ -201,7 +198,6 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
201198
assert(getLogFilesInDirectory(dir).size < logFiles.size)
202199
}
203200
}
204-
205201
// TODO (Hari, TD): Test different failure conditions of writers and readers.
206202
// - Failure while reading incomplete/corrupt file
207203
}
@@ -271,7 +267,8 @@ object WriteAheadLogSuite {
271267
val reader = HdfsUtils.getInputStream(file, hadoopConf)
272268
val buffer = new ArrayBuffer[String]
273269
try {
274-
while (true) { // Read till EOF is thrown
270+
while (true) {
271+
// Read till EOF is thrown
275272
val length = reader.readInt()
276273
val bytes = new Array[Byte](length)
277274
reader.read(bytes)
@@ -294,15 +291,20 @@ object WriteAheadLogSuite {
294291
}
295292

296293
def generateRandomData(numItems: Int = 50, itemSize: Int = 50): Seq[String] = {
297-
(1 to numItems).map { _.toString }
294+
(1 to numItems).map {
295+
_.toString
296+
}
298297
}
299298

300299
def getLogFilesInDirectory(directory: String): Seq[String] = {
301300
val logDirectoryPath = new Path(directory)
302301
val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
303302

303+
implicit def fileStatusOrdering[A <: FileStatus]: Ordering[A] = Ordering
304+
.by(f => f.getModificationTime)
305+
304306
if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) {
305-
fileSystem.listStatus(logDirectoryPath).map {
307+
fileSystem.listStatus(logDirectoryPath).sorted.map {
306308
_.getPath.toString
307309
}
308310
} else {

0 commit comments

Comments
 (0)