-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11419][STREAMING] Parallel recovery for FileBasedWriteAheadLog + minor recovery tweaks #9373
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
573b657
655f4bf
06da0d1
be5a2ab
7f8cfe3
98da092
0b7279f
c2cafe1
83aa28e
9162ed9
c9ea423
c250d2e
285a8d1
22fbaca
f43ecbe
1ba8340
ccf7f5b
7e1829b
dbb31e3
a31822c
79e9b03
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -126,11 +126,11 @@ private[streaming] class FileBasedWriteAheadLog( | |
| val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath | ||
| logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) | ||
|
|
||
| logFilesToRead.iterator.map { file => | ||
| logFilesToRead.par.map { file => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is an expensive operation - you'd end up running an O(n) operation to create a copy (in addition to the copy cost). Do we really need this? I am not entirely sure the copying adds a whole lot of value, considering that this array is not going to be very huge. Also note the additional cost to spin up threads (if the context does not already have them spun up).
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @harishreedharan I did several benchmarks with this. In a setting where you would have a 30 minute window operation, which corresponds to ~9000 files on S3 with
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Which threadpool / execution context is this par operation going to? You must not use the default system execution. |
||
| logDebug(s"Creating log reader with $file") | ||
| val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) | ||
| CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, reader.close _) | ||
| }.flatten.asJava | ||
| }.flatten.toIterator.asJava | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,7 +16,7 @@ | |
| */ | ||
| package org.apache.spark.streaming.util | ||
|
|
||
| import java.io.{Closeable, EOFException} | ||
| import java.io.{IOException, Closeable, EOFException} | ||
| import java.nio.ByteBuffer | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
|
|
@@ -55,6 +55,18 @@ private[streaming] class FileBasedWriteAheadLogReader(path: String, conf: Config | |
| logDebug("Error reading next item, EOF reached", e) | ||
| close() | ||
| false | ||
| case e: IOException => | ||
| logWarning("Error while trying to read data. If the file was deleted, " + | ||
| "this should be okay.", e) | ||
| if (HdfsUtils.checkFileExists(path, conf)) { | ||
| throw e | ||
| } else { | ||
| // file was deleted. This can occur when the daemon cleanup thread takes time to | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. super nit: file -> File, since this comment actually has multiple sentences, its weird to have the first one start with lower case. |
||
| // delete the file during recovery. | ||
| close() | ||
| false | ||
| } | ||
|
|
||
| case e: Exception => | ||
| logWarning("Error while trying to read data from HDFS.", e) | ||
| close() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NPE thrown when streaming context is stopped before recovery is complete