@@ -19,7 +19,7 @@ package org.apache.spark.streaming.util
19
19
import java .io ._
20
20
import java .nio .ByteBuffer
21
21
22
- import org .apache .hadoop .fs .{ FileStatus , Path }
22
+ import org .apache .hadoop .fs .Path
23
23
24
24
import scala .collection .mutable .ArrayBuffer
25
25
import scala .concurrent .duration ._
@@ -31,63 +31,53 @@ import WriteAheadLogSuite._
31
31
import com .google .common .io .Files
32
32
import org .apache .commons .io .FileUtils
33
33
import org .apache .hadoop .conf .Configuration
34
- import org .apache .hadoop .hdfs .MiniDFSCluster
35
- import org .scalatest .{BeforeAndAfterAll , BeforeAndAfter , FunSuite }
34
+ import org .scalatest .{BeforeAndAfter , FunSuite }
36
35
import org .apache .spark .util .Utils
37
36
import org .scalatest .concurrent .Eventually ._
38
37
39
- class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
38
+ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter {
40
39
41
40
val hadoopConf = new Configuration ()
42
- val dfsDir = Files .createTempDir()
43
- val TEST_BUILD_DATA_KEY : String = " test.build.data"
44
- val oldTestBuildDataProp = Option (System .getProperty(TEST_BUILD_DATA_KEY ))
45
- System .setProperty(TEST_BUILD_DATA_KEY , dfsDir.toString)
46
- val cluster = new MiniDFSCluster (new Configuration , 2 , true , null )
47
- val nnPort = cluster.getNameNode.getNameNodeAddress.getPort
48
- val hdfsUrl = s " hdfs://localhost: $nnPort/ ${getRandomString()}/ "
49
- var pathForTest : String = null
50
-
51
- override def beforeAll () {
52
- cluster.waitActive()
53
- }
41
+ var tempDir : File = null
42
+ var dirForTest : String = null
43
+ var fileForTest : String = null
54
44
55
45
before {
56
- pathForTest = hdfsUrl + getRandomString()
46
+ tempDir = Files .createTempDir()
47
+ dirForTest = " file:///" + tempDir.toString
48
+ fileForTest = " file:///" + new File (tempDir, getRandomString()).toString
57
49
}
58
50
59
- override def afterAll () {
60
- cluster.shutdown()
61
- FileUtils .deleteDirectory(dfsDir)
62
- oldTestBuildDataProp.foreach(System .setProperty(TEST_BUILD_DATA_KEY , _))
51
+ after {
52
+ FileUtils .deleteDirectory(tempDir)
63
53
}
64
54
65
55
test(" WriteAheadLogWriter - writing data" ) {
66
56
val dataToWrite = generateRandomData()
67
- val writer = new WriteAheadLogWriter (pathForTest , hadoopConf)
57
+ val writer = new WriteAheadLogWriter (fileForTest , hadoopConf)
68
58
val segments = dataToWrite.map(data => writer.write(data))
69
59
writer.close()
70
- val writtenData = readDataManually(pathForTest , segments)
60
+ val writtenData = readDataManually(fileForTest , segments)
71
61
assert(writtenData.toArray === dataToWrite.toArray)
72
62
}
73
63
74
64
test(" WriteAheadLogWriter - syncing of data by writing and reading immediately" ) {
75
65
val dataToWrite = generateRandomData()
76
- val writer = new WriteAheadLogWriter (pathForTest , hadoopConf)
66
+ val writer = new WriteAheadLogWriter (fileForTest , hadoopConf)
77
67
dataToWrite.foreach { data =>
78
- val segment = writer.write(ByteBuffer .wrap (data.getBytes() ))
79
- val reader = new WriteAheadLogRandomReader (pathForTest , hadoopConf)
68
+ val segment = writer.write(stringToByteBuffer (data))
69
+ val reader = new WriteAheadLogRandomReader (fileForTest , hadoopConf)
80
70
val dataRead = reader.read(segment)
81
- assert(data === new String (dataRead.array() ))
71
+ assert(data === byteBufferToString (dataRead))
82
72
}
83
73
writer.close()
84
74
}
85
75
86
76
test(" WriteAheadLogReader - sequentially reading data" ) {
87
77
// Write data manually for testing the sequential reader
88
78
val writtenData = generateRandomData()
89
- writeDataManually(writtenData, pathForTest )
90
- val reader = new WriteAheadLogReader (pathForTest , hadoopConf)
79
+ writeDataManually(writtenData, fileForTest )
80
+ val reader = new WriteAheadLogReader (fileForTest , hadoopConf)
91
81
val readData = reader.toSeq.map(byteBufferToString)
92
82
assert(readData.toList === writtenData.toList)
93
83
assert(reader.hasNext === false )
@@ -100,9 +90,9 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
100
90
test(" WriteAheadLogReader - sequentially reading data written with writer" ) {
101
91
// Write data manually for testing the sequential reader
102
92
val dataToWrite = generateRandomData()
103
- writeDataUsingWriter(pathForTest , dataToWrite)
93
+ writeDataUsingWriter(fileForTest , dataToWrite)
104
94
val iter = dataToWrite.iterator
105
- val reader = new WriteAheadLogReader (pathForTest , hadoopConf)
95
+ val reader = new WriteAheadLogReader (fileForTest , hadoopConf)
106
96
reader.foreach { byteBuffer =>
107
97
assert(byteBufferToString(byteBuffer) === iter.next())
108
98
}
@@ -112,11 +102,11 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
112
102
test(" WriteAheadLogRandomReader - reading data using random reader" ) {
113
103
// Write data manually for testing the random reader
114
104
val writtenData = generateRandomData()
115
- val segments = writeDataManually(writtenData, pathForTest )
105
+ val segments = writeDataManually(writtenData, fileForTest )
116
106
117
107
// Get a random order of these segments and read them back
118
108
val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10 ).flatten
119
- val reader = new WriteAheadLogRandomReader (pathForTest , hadoopConf)
109
+ val reader = new WriteAheadLogRandomReader (fileForTest , hadoopConf)
120
110
writtenDataAndSegments.foreach { case (data, segment) =>
121
111
assert(data === byteBufferToString(reader.read(segment)))
122
112
}
@@ -126,11 +116,11 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
126
116
test(" WriteAheadLogRandomReader - reading data using random reader written with writer" ) {
127
117
// Write data using writer for testing the random reader
128
118
val data = generateRandomData()
129
- val segments = writeDataUsingWriter(pathForTest , data)
119
+ val segments = writeDataUsingWriter(fileForTest , data)
130
120
131
121
// Read a random sequence of segments and verify read data
132
122
val dataAndSegments = data.zip(segments).toSeq.permutations.take(10 ).flatten
133
- val reader = new WriteAheadLogRandomReader (pathForTest , hadoopConf)
123
+ val reader = new WriteAheadLogRandomReader (fileForTest , hadoopConf)
134
124
dataAndSegments.foreach { case (data, segment) =>
135
125
assert(data === byteBufferToString(reader.read(segment)))
136
126
}
@@ -140,62 +130,58 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter with BeforeAndAfte
140
130
test(" WriteAheadLogManager - write rotating logs" ) {
141
131
// Write data using manager
142
132
val dataToWrite = generateRandomData()
143
- val dir = pathForTest
144
- writeDataUsingManager(dir, dataToWrite)
133
+ writeDataUsingManager(dirForTest, dataToWrite)
145
134
146
135
// Read data manually to verify the written data
147
- val logFiles = getLogFilesInDirectory(dir )
136
+ val logFiles = getLogFilesInDirectory(dirForTest )
148
137
assert(logFiles.size > 1 )
149
138
val writtenData = logFiles.flatMap { file => readDataManually(file)}
150
139
assert(writtenData.toList === dataToWrite.toList)
151
140
}
152
141
153
142
test(" WriteAheadLogManager - read rotating logs" ) {
154
143
// Write data manually for testing reading through manager
155
- val dir = pathForTest
156
144
val writtenData = (1 to 10 ).map { i =>
157
145
val data = generateRandomData()
158
- val file = dir + s " /log- $i- $i"
146
+ val file = dirForTest + s " /log- $i- $i"
159
147
writeDataManually(data, file)
160
148
data
161
149
}.flatten
162
150
163
- val logDirectoryPath = new Path (dir )
151
+ val logDirectoryPath = new Path (dirForTest )
164
152
val fileSystem = HdfsUtils .getFileSystemForPath(logDirectoryPath, hadoopConf)
165
153
assert(fileSystem.exists(logDirectoryPath) === true )
166
154
167
155
// Read data using manager and verify
168
- val readData = readDataUsingManager(dir )
156
+ val readData = readDataUsingManager(dirForTest )
169
157
assert(readData.toList === writtenData.toList)
170
158
}
171
159
172
160
test(" WriteAheadLogManager - recover past logs when creating new manager" ) {
173
161
// Write data with manager, recover with new manager and verify
174
162
val dataToWrite = generateRandomData()
175
- val dir = pathForTest
176
- writeDataUsingManager(dir, dataToWrite)
177
- val logFiles = getLogFilesInDirectory(dir)
163
+ writeDataUsingManager(dirForTest, dataToWrite)
164
+ val logFiles = getLogFilesInDirectory(dirForTest)
178
165
assert(logFiles.size > 1 )
179
- val readData = readDataUsingManager(dir )
166
+ val readData = readDataUsingManager(dirForTest )
180
167
assert(dataToWrite.toList === readData.toList)
181
168
}
182
169
183
170
test(" WriteAheadLogManager - cleanup old logs" ) {
184
171
// Write data with manager, recover with new manager and verify
185
- val dir = pathForTest
186
172
val dataToWrite = generateRandomData()
187
173
val fakeClock = new ManualClock
188
- val manager = new WriteAheadLogManager (dir , hadoopConf,
174
+ val manager = new WriteAheadLogManager (dirForTest , hadoopConf,
189
175
rollingIntervalSecs = 1 , callerName = " WriteAheadLogSuite" , clock = fakeClock)
190
176
dataToWrite.foreach { item =>
191
177
fakeClock.addToTime(500 ) // half second for each
192
178
manager.writeToLog(item)
193
179
}
194
- val logFiles = getLogFilesInDirectory(dir )
180
+ val logFiles = getLogFilesInDirectory(dirForTest )
195
181
assert(logFiles.size > 1 )
196
182
manager.cleanupOldLogs(fakeClock.currentTime() / 2 )
197
183
eventually(timeout(1 second), interval(10 milliseconds)) {
198
- assert(getLogFilesInDirectory(dir ).size < logFiles.size)
184
+ assert(getLogFilesInDirectory(dirForTest ).size < logFiles.size)
199
185
}
200
186
}
201
187
// TODO (Hari, TD): Test different failure conditions of writers and readers.
@@ -305,7 +291,7 @@ object WriteAheadLogSuite {
305
291
if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) {
306
292
fileSystem.listStatus(logDirectoryPath).map {
307
293
_.getPath.toString
308
- }
294
+ }.sorted
309
295
} else {
310
296
Seq .empty
311
297
}
0 commit comments