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 @@ -42,18 +42,18 @@ import org.apache.spark.sql.SparkSession
* line 1: version
* line 2: metadata (optional json string)
*/
class BatchCommitLog(sparkSession: SparkSession, path: String)
class CommitLog(sparkSession: SparkSession, path: String)
extends HDFSMetadataLog[String](sparkSession, path) {

import BatchCommitLog._
import CommitLog._

def add(batchId: Long): Unit = {
super.add(batchId, EMPTY_JSON)
}

override def add(batchId: Long, metadata: String): Boolean = {
throw new UnsupportedOperationException(
"BatchCommitLog does not take any metadata, use 'add(batchId)' instead")
"CommitLog does not take any metadata, use 'add(batchId)' instead")
}

override protected def deserialize(in: InputStream): String = {
Expand All @@ -76,7 +76,7 @@ class BatchCommitLog(sparkSession: SparkSession, path: String)
}
}

object BatchCommitLog {
object CommitLog {
private val VERSION = 1
private val EMPTY_JSON = "{}"
}
Expand Down

Large diffs are not rendered by default.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
"is not supported in streaming DataFrames/Datasets and will be disabled.")
}

new StreamingQueryWrapper(new StreamExecution(
new StreamingQueryWrapper(new MicroBatchExecution(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we make it class name configurable?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I'm not sure what you have in mind here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean, how we switch between ContinuousExecution and MicroBatchExecution?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My current thinking is to have it be a new trigger type. It can't really be a config, because continuous processing (at least in the initial implementation) won't support all operators.

sparkSession,
userSpecifiedName.orNull,
checkpointLocation,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,8 +260,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche
CheckLastBatch((10, 5)),
StopStream,
AssertOnQuery { q => // purge commit and clear the sink
val commit = q.batchCommitLog.getLatest().map(_._1).getOrElse(-1L) + 1L
q.batchCommitLog.purge(commit)
val commit = q.commitLog.getLatest().map(_._1).getOrElse(-1L) + 1L
q.commitLog.purge(commit)
q.sink.asInstanceOf[MemorySink].clear()
true
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1024,7 +1024,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
expectedCompactInterval: Int): Boolean = {
import CompactibleFileStreamLog._

val fileSource = (execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource]
val fileSource = getSourcesFromStreamingQuery(execution).head
val metadataLog = fileSource invokePrivate _metadataLog()

if (isCompactionBatch(batchId, expectedCompactInterval)) {
Expand Down Expand Up @@ -1100,8 +1100,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
CheckAnswer("keep1", "keep2", "keep3"),
AssertOnQuery("check getBatch") { execution: StreamExecution =>
val _sources = PrivateMethod[Seq[Source]]('sources)
val fileSource =
(execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource]
val fileSource = getSourcesFromStreamingQuery(execution).head

def verify(startId: Option[Int], endId: Int, expected: String*): Unit = {
val start = startId.map(new FileStreamSourceOffset(_))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ class StreamSuite extends StreamTest {

// Check the latest batchid in the commit log
def CheckCommitLogLatestBatchId(expectedId: Int): AssertOnQuery =
AssertOnQuery(_.batchCommitLog.getLatest().get._1 == expectedId,
AssertOnQuery(_.commitLog.getLatest().get._1 == expectedId,
s"commitLog's latest should be $expectedId")

// Ensure that there has not been an incremental execution after restart
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,12 +300,14 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be
if (currentStream != null) currentStream.committedOffsets.toString else "not started"

def threadState =
if (currentStream != null && currentStream.microBatchThread.isAlive) "alive" else "dead"
def threadStackTrace = if (currentStream != null && currentStream.microBatchThread.isAlive) {
s"Thread stack trace: ${currentStream.microBatchThread.getStackTrace.mkString("\n")}"
} else {
""
}
if (currentStream != null && currentStream.queryExecutionThread.isAlive) "alive" else "dead"

def threadStackTrace =
if (currentStream != null && currentStream.queryExecutionThread.isAlive) {
s"Thread stack trace: ${currentStream.queryExecutionThread.getStackTrace.mkString("\n")}"
} else {
""
}

def testState =
s"""
Expand Down Expand Up @@ -460,7 +462,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be
verify(currentStream != null, "can not stop a stream that is not running")
try failAfter(streamingTimeout) {
currentStream.stop()
verify(!currentStream.microBatchThread.isAlive,
verify(!currentStream.queryExecutionThread.isAlive,
s"microbatch thread not stopped")
verify(!currentStream.isActive,
"query.isActive() is false even after stopping")
Expand All @@ -486,7 +488,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be
currentStream.awaitTermination()
}
eventually("microbatch thread not stopped after termination with failure") {
assert(!currentStream.microBatchThread.isAlive)
assert(!currentStream.queryExecutionThread.isAlive)
}
verify(currentStream.exception === Some(thrownException),
s"incorrect exception returned by query.exception()")
Expand Down Expand Up @@ -614,7 +616,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be
case e: org.scalatest.exceptions.TestFailedDueToTimeoutException =>
failTest("Timed out waiting for stream", e)
} finally {
if (currentStream != null && currentStream.microBatchThread.isAlive) {
if (currentStream != null && currentStream.queryExecutionThread.isAlive) {
currentStream.stop()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest
StopStream,
AssertOnQuery { q => // clear the sink
q.sink.asInstanceOf[MemorySink].clear()
q.batchCommitLog.purge(3)
q.commitLog.purge(3)
// advance by a minute i.e., 90 seconds total
clock.advance(60 * 1000L)
true
Expand Down Expand Up @@ -352,7 +352,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest
StopStream,
AssertOnQuery { q => // clear the sink
q.sink.asInstanceOf[MemorySink].clear()
q.batchCommitLog.purge(3)
q.commitLog.purge(3)
// advance by 60 days i.e., 90 days total
clock.advance(DateTimeUtils.MILLIS_PER_DAY * 60)
true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -173,12 +173,12 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
StopStream, // clears out StreamTest state
AssertOnQuery { q =>
// both commit log and offset log contain the same (latest) batch id
q.batchCommitLog.getLatest().map(_._1).getOrElse(-1L) ==
q.commitLog.getLatest().map(_._1).getOrElse(-1L) ==
q.offsetLog.getLatest().map(_._1).getOrElse(-2L)
},
AssertOnQuery { q =>
// blow away commit log and sink result
q.batchCommitLog.purge(1)
q.commitLog.purge(1)
q.sink.asInstanceOf[MemorySink].clear()
true
},
Expand Down