Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/9373#discussion_r44615768
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
---
@@ -146,30 +154,32 @@ private[streaming] class FileBasedWriteAheadLog(
* asynchronously.
*/
def clean(threshTime: Long, waitForCompletion: Boolean): Unit = {
- val oldLogFiles = synchronized { pastLogs.filter { _.endTime <
threshTime } }
+ val oldLogFiles = synchronized {
+ val expiredLogs = pastLogs.filter { _.endTime < threshTime }
+ pastLogs --= expiredLogs
+ expiredLogs
+ }
logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in
$logDirectory " +
s"older than $threshTime: ${oldLogFiles.map { _.path
}.mkString("\n")}")
-
- def deleteFiles() {
- oldLogFiles.foreach { logInfo =>
- try {
- val path = new Path(logInfo.path)
- val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf)
- fs.delete(path, true)
- synchronized { pastLogs -= logInfo }
- logDebug(s"Cleared log file $logInfo")
- } catch {
- case ex: Exception =>
- logWarning(s"Error clearing write ahead log file $logInfo", ex)
- }
+ def deleteFile(walInfo: LogInfo): Unit = {
+ try {
+ val path = new Path(walInfo.path)
+ val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf)
+ fs.delete(path, true)
+ logDebug(s"Cleared log file $walInfo")
+ } catch {
+ case ex: Exception =>
+ logWarning(s"Error clearing write ahead log file $walInfo", ex)
}
logInfo(s"Cleared log files in $logDirectory older than $threshTime")
}
- if (!executionContext.isShutdown) {
- val f = Future { deleteFiles() }
- if (waitForCompletion) {
- import scala.concurrent.duration._
- Await.ready(f, 1 second)
+ oldLogFiles.foreach { logInfo =>
--- End diff --
nit: When waitForCompletion is true, this whole deletion is done one by
one.
Might be better to compose all the Futures into a single Future to wait.
Not a problem really as waitForCompletion is true only in tests. Do this
only if there is any other critical feedback.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]