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 @@ -354,7 +354,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
CheckAnswer("a", "b"),

// SLeeps longer than 5ms (maxFileAge)
AssertOnQuery { _ => Thread.sleep(10); true },
// Unfortunately since a lot of file system does not have modification time granularity
// finer grained than 1 sec, we need to use 1 sec here.
AssertOnQuery { _ => Thread.sleep(1000); true },

AddTextFileData("c\nd", src, tmp),
CheckAnswer("a", "b", "c", "d"),
Expand All @@ -363,7 +365,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
val source = streamExecution.logicalPlan.collect { case e: StreamingExecutionRelation =>
e.source.asInstanceOf[FileStreamSource]
}.head
source.seenFiles.size == 1
assert(source.seenFiles.size == 1)
true
}
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -188,8 +188,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts {
new AssertOnQuery(condition, message)
}

def apply(message: String)(condition: StreamExecution => Unit): AssertOnQuery = {
new AssertOnQuery(s => { condition(s); true }, message)
def apply(message: String)(condition: StreamExecution => Boolean): AssertOnQuery = {
new AssertOnQuery(condition, message)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
// No progress events or termination events
assert(listener.progressStatuses.isEmpty)
assert(listener.terminationStatus === null)
true
},
AddDataMemory(input, Seq(1, 2, 3)),
CheckAnswer(1, 2, 3),
Expand All @@ -84,6 +85,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
// No termination events
assert(listener.terminationStatus === null)
}
true
},
StopStream,
AssertOnQuery("Incorrect query status in onQueryTerminated") { query =>
Expand All @@ -97,6 +99,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
assert(listener.terminationException === None)
}
listener.checkAsyncErrors()
true
}
)
}
Expand Down