@@ -27,7 +27,8 @@ import scala.collection.JavaConverters._
27
27
import scala .collection .mutable
28
28
29
29
import com .google .common .io .Files
30
- import org .apache .hadoop .fs .Path
30
+ import org .apache .commons .io .IOUtils
31
+ import org .apache .hadoop .fs .{FileSystem , Path }
31
32
import org .apache .hadoop .io .{LongWritable , Text }
32
33
import org .apache .hadoop .mapreduce .lib .input .TextInputFormat
33
34
import org .scalatest .BeforeAndAfter
@@ -130,10 +131,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
130
131
}
131
132
132
133
test(" binary records stream" ) {
133
- var testDir : File = null
134
- try {
134
+ withTempDir { testDir =>
135
135
val batchDuration = Seconds (2 )
136
- testDir = Utils .createTempDir()
137
136
// Create a file that exists before the StreamingContext is created:
138
137
val existingFile = new File (testDir, " 0" )
139
138
Files .write(" 0\n " , existingFile, StandardCharsets .UTF_8 )
@@ -176,8 +175,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
176
175
assert(obtainedOutput(i) === input.map(b => (b + i).toByte))
177
176
}
178
177
}
179
- } finally {
180
- if (testDir != null ) Utils .deleteRecursively(testDir)
181
178
}
182
179
}
183
180
@@ -190,10 +187,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
190
187
}
191
188
192
189
test(" file input stream - wildcard" ) {
193
- var testDir : File = null
194
- try {
190
+ withTempDir { testDir =>
195
191
val batchDuration = Seconds (2 )
196
- testDir = Utils .createTempDir()
197
192
val testSubDir1 = Utils .createDirectory(testDir.toString, " tmp1" )
198
193
val testSubDir2 = Utils .createDirectory(testDir.toString, " tmp2" )
199
194
@@ -221,12 +216,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
221
216
// not enough to trigger a batch
222
217
clock.advance(batchDuration.milliseconds / 2 )
223
218
224
- def createFileAndAdvenceTime (data : Int , dir : File ): Unit = {
219
+ def createFileAndAdvanceTime (data : Int , dir : File ): Unit = {
225
220
val file = new File (testSubDir1, data.toString)
226
221
Files .write(data + " \n " , file, StandardCharsets .UTF_8 )
227
222
assert(file.setLastModified(clock.getTimeMillis()))
228
223
assert(file.lastModified === clock.getTimeMillis())
229
- logInfo(" Created file " + file)
224
+ logInfo(s " Created file $ file" )
230
225
// Advance the clock after creating the file to avoid a race when
231
226
// setting its modification time
232
227
clock.advance(batchDuration.milliseconds)
@@ -236,18 +231,85 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
236
231
}
237
232
// Over time, create files in the temp directory 1
238
233
val input1 = Seq (1 , 2 , 3 , 4 , 5 )
239
- input1.foreach(i => createFileAndAdvenceTime (i, testSubDir1))
234
+ input1.foreach(i => createFileAndAdvanceTime (i, testSubDir1))
240
235
241
236
// Over time, create files in the temp directory 1
242
237
val input2 = Seq (6 , 7 , 8 , 9 , 10 )
243
- input2.foreach(i => createFileAndAdvenceTime (i, testSubDir2))
238
+ input2.foreach(i => createFileAndAdvanceTime (i, testSubDir2))
244
239
245
240
// Verify that all the files have been read
246
241
val expectedOutput = (input1 ++ input2).map(_.toString).toSet
247
242
assert(outputQueue.asScala.flatten.toSet === expectedOutput)
248
243
}
249
- } finally {
250
- if (testDir != null ) Utils .deleteRecursively(testDir)
244
+ }
245
+ }
246
+
247
+ test(" Modified files are correctly detected." ) {
248
+ withTempDir { testDir =>
249
+ val batchDuration = Seconds (2 )
250
+ val durationMs = batchDuration.milliseconds
251
+ val testPath = new Path (testDir.toURI)
252
+ val streamDir = new Path (testPath, " streaming" )
253
+ val streamGlobPath = new Path (streamDir, " sub*" )
254
+ val generatedDir = new Path (testPath, " generated" )
255
+ val generatedSubDir = new Path (generatedDir, " subdir" )
256
+ val renamedSubDir = new Path (streamDir, " subdir" )
257
+
258
+ withStreamingContext(new StreamingContext (conf, batchDuration)) { ssc =>
259
+ val sparkContext = ssc.sparkContext
260
+ val hc = sparkContext.hadoopConfiguration
261
+ val fs = FileSystem .get(testPath.toUri, hc)
262
+
263
+ fs.delete(testPath, true )
264
+ fs.mkdirs(testPath)
265
+ fs.mkdirs(streamDir)
266
+ fs.mkdirs(generatedSubDir)
267
+
268
+ def write (path : Path , text : String ): Unit = {
269
+ val out = fs.create(path, true )
270
+ IOUtils .write(text, out)
271
+ out.close()
272
+ }
273
+
274
+ val clock = ssc.scheduler.clock.asInstanceOf [ManualClock ]
275
+ val existingFile = new Path (generatedSubDir, " existing" )
276
+ write(existingFile, " existing\n " )
277
+ val status = fs.getFileStatus(existingFile)
278
+ clock.setTime(status.getModificationTime + durationMs)
279
+ val batchCounter = new BatchCounter (ssc)
280
+ val fileStream = ssc.textFileStream(streamGlobPath.toUri.toString)
281
+ val outputQueue = new ConcurrentLinkedQueue [Seq [String ]]
282
+ val outputStream = new TestOutputStream (fileStream, outputQueue)
283
+ outputStream.register()
284
+
285
+ ssc.start()
286
+ clock.advance(durationMs)
287
+ eventually(eventuallyTimeout) {
288
+ assert(1 === batchCounter.getNumCompletedBatches)
289
+ }
290
+ // create and rename the file
291
+ // put a file into the generated directory
292
+ val textPath = new Path (generatedSubDir, " renamed.txt" )
293
+ write(textPath, " renamed\n " )
294
+ val now = clock.getTimeMillis()
295
+ val modTime = now + durationMs / 2
296
+ fs.setTimes(textPath, modTime, modTime)
297
+ val textFilestatus = fs.getFileStatus(existingFile)
298
+ assert(textFilestatus.getModificationTime < now + durationMs)
299
+
300
+ // rename the directory under the path being scanned
301
+ fs.rename(generatedSubDir, renamedSubDir)
302
+
303
+ // move forward one window
304
+ clock.advance(durationMs)
305
+ // await the next scan completing
306
+ eventually(eventuallyTimeout) {
307
+ assert(2 === batchCounter.getNumCompletedBatches)
308
+ }
309
+ // verify that the "renamed" file is found, but not the "existing" one which is out of
310
+ // the window
311
+ assert(Set (" renamed" ) === outputQueue.asScala.flatten.toSet)
312
+ }
251
313
}
252
314
}
253
315
@@ -416,10 +478,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
416
478
}
417
479
418
480
def testFileStream (newFilesOnly : Boolean ) {
419
- var testDir : File = null
420
- try {
481
+ withTempDir { testDir =>
421
482
val batchDuration = Seconds (2 )
422
- testDir = Utils .createTempDir()
423
483
// Create a file that exists before the StreamingContext is created:
424
484
val existingFile = new File (testDir, " 0" )
425
485
Files .write(" 0\n " , existingFile, StandardCharsets .UTF_8 )
@@ -466,8 +526,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
466
526
}
467
527
assert(outputQueue.asScala.flatten.toSet === expectedOutput)
468
528
}
469
- } finally {
470
- if (testDir != null ) Utils .deleteRecursively(testDir)
471
529
}
472
530
}
473
531
}
0 commit comments