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 @@ -68,19 +68,16 @@ class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Seria
}
datasetWithIncrementalExecution.foreachPartition { iter =>
if (writer.open(TaskContext.getPartitionId(), batchId)) {
var isFailed = false
try {
while (iter.hasNext) {
writer.process(iter.next())
}
} catch {
case e: Throwable =>
Copy link
Contributor

@tdas tdas Oct 28, 2016

Choose a reason for hiding this comment

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

wait.. isnt it simpler to do

try {
  ...
} catch {
 case e => 
    logError(.....)
    throw e
} finally {
   writer.close(e)
}

Copy link
Member Author

Choose a reason for hiding this comment

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

cannot access e inside finally.

Copy link
Contributor

Choose a reason for hiding this comment

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

oops. my bad.

isFailed = true
writer.close(e)
throw e
}
if (!isFailed) {
writer.close(null)
}
writer.close(null)
} else {
writer.close(null)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,9 @@ import scala.collection.mutable

import org.scalatest.BeforeAndAfter

import org.apache.spark.SparkException
import org.apache.spark.sql.ForeachWriter
import org.apache.spark.sql.streaming.{OutputMode, StreamTest}
import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest}
import org.apache.spark.sql.test.SharedSQLContext

class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter {
Expand Down Expand Up @@ -136,7 +137,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf
}
}

test("foreach with error") {
testQuietly("foreach with error") {
withTempDir { checkpointDir =>
val input = MemoryStream[Int]
val query = input.toDS().repartition(1).writeStream
Expand All @@ -148,16 +149,24 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf
}
}).start()
input.addData(1, 2, 3, 4)
query.processAllAvailable()

// Error in `process` should fail the Spark job
val e = intercept[StreamingQueryException] {
query.processAllAvailable()
}
assert(e.getCause.isInstanceOf[SparkException])
assert(e.getCause.getCause.getMessage === "error")
assert(query.isActive === false)

val allEvents = ForeachSinkSuite.allEvents()
assert(allEvents.size === 1)
assert(allEvents(0)(0) === ForeachSinkSuite.Open(partition = 0, version = 0))
assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1))
assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1))

// `close` should be called with the error
val errorEvent = allEvents(0)(2).asInstanceOf[ForeachSinkSuite.Close]
assert(errorEvent.error.get.isInstanceOf[RuntimeException])
assert(errorEvent.error.get.getMessage === "error")
query.stop()
}
}
}
Expand Down