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 @@ -169,7 +169,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag](
*/
private def compact(batchId: Long, logs: Array[T]): Boolean = {
val validBatches = getValidBatchesBeforeCompactionBatch(batchId, compactInterval)
val allLogs = validBatches.flatMap(batchId => super.get(batchId)).flatten ++ logs
val allLogs = validBatches.map { id =>
super.get(id).getOrElse {
throw new IllegalStateException(
s"${batchIdToPath(id)} doesn't exist when compacting batch $batchId " +
s"(compactInterval: $compactInterval)")
}
}.flatten ++ logs
// Return false as there is another writer.
super.add(batchId, compactLogs(allLogs).toArray)
}
Expand All @@ -186,7 +192,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag](
if (latestId >= 0) {
try {
val logs =
getAllValidBatches(latestId, compactInterval).flatMap(id => super.get(id)).flatten
getAllValidBatches(latestId, compactInterval).map { id =>
super.get(id).getOrElse {
throw new IllegalStateException(
s"${batchIdToPath(id)} doesn't exist " +
s"(latestId: $latestId, compactInterval: $compactInterval)")
}
}.flatten
return compactLogs(logs).toArray
} catch {
case e: IOException =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,10 @@ class FileStreamSourceLog(
Map.empty[Long, Option[Array[FileEntry]]]
}

(existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1)
val batches =
(existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1)
HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId)
batches
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path:
serialize(metadata, output)
return Some(tempPath)
} finally {
IOUtils.closeQuietly(output)
output.close()
Copy link
Member Author

Choose a reason for hiding this comment

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

The output stream may fail to close (e.g., fail to flush the internal buffer), if it happens, we should fail the query rather than ignoring it.

}
} catch {
case e: FileAlreadyExistsException =>
Expand Down Expand Up @@ -211,13 +211,17 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path:
}

override def get(startId: Option[Long], endId: Option[Long]): Array[(Long, T)] = {
assert(startId.isEmpty || endId.isEmpty || startId.get <= endId.get)
val files = fileManager.list(metadataPath, batchFilesFilter)
val batchIds = files
.map(f => pathToBatchId(f.getPath))
.filter { batchId =>
(endId.isEmpty || batchId <= endId.get) && (startId.isEmpty || batchId >= startId.get)
}
batchIds.sorted.map(batchId => (batchId, get(batchId))).filter(_._2.isDefined).map {
}.sorted

verifyBatchIds(batchIds, startId, endId)

batchIds.map(batchId => (batchId, get(batchId))).filter(_._2.isDefined).map {
case (batchId, metadataOption) =>
(batchId, metadataOption.get)
}
Expand Down Expand Up @@ -437,4 +441,51 @@ object HDFSMetadataLog {
}
}
}

/**
* Verify if batchIds are continuous and between `startId` and `endId`.
*
* @param batchIds the sorted ids to verify.
* @param startId the start id. If it's set, batchIds should start with this id.
* @param endId the start id. If it's set, batchIds should end with this id.
*/
def verifyBatchIds(batchIds: Seq[Long], startId: Option[Long], endId: Option[Long]): Unit = {
// Verify that we can get all batches between `startId` and `endId`.
if (startId.isDefined || endId.isDefined) {
if (batchIds.isEmpty) {
throw new IllegalStateException(s"batch ${startId.orElse(endId).get} doesn't exist")
Copy link
Contributor

Choose a reason for hiding this comment

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

would be good to print the range that was asked for. otherwise its hard to see what was expected while debugging.

}
if (startId.isDefined) {
val minBatchId = batchIds.head
assert(minBatchId >= startId.get)
if (minBatchId != startId.get) {
val missingBatchIds = startId.get to minBatchId
throw new IllegalStateException(
s"batches (${missingBatchIds.mkString(", ")}) don't exist " +
s"(startId: $startId, endId: $endId)")
}
}

if (endId.isDefined) {
val maxBatchId = batchIds.last
assert(maxBatchId <= endId.get)
if (maxBatchId != endId.get) {
val missingBatchIds = maxBatchId to endId.get
throw new IllegalStateException(
s"batches (${missingBatchIds.mkString(", ")}) don't exist " +
s"(startId: $startId, endId: $endId)")
}
}
}

if (batchIds.nonEmpty) {
val minBatchId = batchIds.head
val maxBatchId = batchIds.last
val missingBatchIds = (minBatchId to maxBatchId).toSet -- batchIds
if (missingBatchIds.nonEmpty) {
throw new IllegalStateException(s"batches (${missingBatchIds.mkString(", ")}) " +
s"don't exist (startId: $startId, endId: $endId)")
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -438,7 +438,10 @@ class StreamExecution(
availableOffsets = nextOffsets.toStreamProgress(sources)
/* Initialize committed offsets to a committed batch, which at this
* is the second latest batch id in the offset log. */
offsetLog.get(latestBatchId - 1).foreach { secondLatestBatchId =>
if (latestBatchId != 0) {
val secondLatestBatchId = offsetLog.get(latestBatchId - 1).getOrElse {
throw new IllegalStateException(s"batch ${latestBatchId - 1} doesn't exist")
}
committedOffsets = secondLatestBatchId.toStreamProgress(sources)
}

Expand Down Expand Up @@ -565,10 +568,14 @@ class StreamExecution(

// Now that we've updated the scheduler's persistent checkpoint, it is safe for the
// sources to discard data from the previous batch.
val prevBatchOff = offsetLog.get(currentBatchId - 1)
if (prevBatchOff.isDefined) {
prevBatchOff.get.toStreamProgress(sources).foreach {
case (src, off) => src.commit(off)
if (currentBatchId != 0) {
val prevBatchOff = offsetLog.get(currentBatchId - 1)
if (prevBatchOff.isDefined) {
prevBatchOff.get.toStreamProgress(sources).foreach {
case (src, off) => src.commit(off)
}
} else {
throw new IllegalStateException(s"batch $currentBatchId doesn't exist")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,23 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext {
fm.rename(path2, path3)
}
}

test("verifyBatchIds") {
import HDFSMetadataLog.verifyBatchIds
verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), Some(3L))
verifyBatchIds(Seq(1L), Some(1L), Some(1L))
Copy link
Contributor

Choose a reason for hiding this comment

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

you didnt test the valid cases when one of the start or end is None.

verifyBatchIds(Seq(1L, 2L, 3L), None, Some(3L))
verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), None)
verifyBatchIds(Seq(1L, 2L, 3L), None, None)

intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), None))
intercept[IllegalStateException](verifyBatchIds(Seq(), None, Some(1L)))
intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), Some(1L)))
intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), None))
intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L)))
intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L)))
intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L)))
}
}

/** FakeFileSystem to test fallback of the HDFSMetadataLog from FileContext to FileSystem API */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1314,6 +1314,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
val metadataLog =
new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath)
assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0))))
assert(metadataLog.add(1, Array(FileEntry(s"$scheme:///file2", 200L, 0))))
Copy link
Contributor

Choose a reason for hiding this comment

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

what was this change for?

Copy link
Member Author

Choose a reason for hiding this comment

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

newSource.getBatch(None, FileStreamSourceOffset(1)) will fail without this because batch 1 doesn't exist.


val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil,
dir.getAbsolutePath, Map.empty)
Expand Down