Skip to content
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

[SPARK-4026][Streaming] Write ahead log management #2882

Closed
wants to merge 20 commits into from

Conversation

tdas
Copy link
Contributor

@tdas tdas commented Oct 21, 2014

As part of the effort to avoid data loss on Spark Streaming driver failure, we want to implement a write ahead log that can write received data to HDFS. This allows the received data to be persist across driver failures. So when the streaming driver is restarted, it can find and reprocess all the data that were received but not processed.

This was primarily implemented by @harishreedharan. This is still WIP, as he is going to improve the unitests by using HDFS mini cluster.

@tdas
Copy link
Contributor Author

tdas commented Oct 21, 2014

Please review this @JoshRosen

@SparkQA
Copy link

SparkQA commented Oct 21, 2014

QA tests have started for PR 2882 at commit 5182ffb.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 21, 2014

QA tests have finished for PR 2882 at commit 5182ffb.

  • This patch fails RAT tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22002/
Test FAILed.

@tdas tdas changed the title [SPARK-4026][Streaming] synchronously write received data to HDFS and recover on driver failure [SPARK-4026][Streaming] Write ahead log management Oct 21, 2014
@SparkQA
Copy link

SparkQA commented Oct 21, 2014

QA tests have started for PR 2882 at commit 4ab602a.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 21, 2014

QA tests have finished for PR 2882 at commit 4ab602a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22006/
Test PASSed.

*/
private[streaming] class WriteAheadLogWriter(path: String, hadoopConf: Configuration)
extends Closeable {
private val underlyingStream: Either[DataOutputStream, FSDataOutputStream] = {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

WIP: this file is going to be updated by @harishreedharan to get rid of the local file customizations.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah, that makes sense. I guess you can still use the HDFS API to write to local files for testing purposes.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yep. And for all tests, we are just going to use Hadoop Minicluster anyway.


private lazy val hadoopFlushMethod = {
val cls = classOf[FSDataOutputStream]
Try(cls.getMethod("hflush")).orElse(Try(cls.getMethod("sync"))).toOption
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice Scala one-liner :)

Why do we need this reflection, though? Is this necessary to support multiple Hadoop versions? If so, could you add a one-line comment to explain this?

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually we do, since Spark supports Hadoop 1 to Hadoop 2.5.0 right now. In Hadoop 1.x, the "sync" method did the same thing hflush does in 2.5.0 - so in short we do.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@SparkQA
Copy link

SparkQA commented Oct 23, 2014

QA tests have finished for PR 2882 at commit 3881706.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22067/
Test PASSed.

@SparkQA
Copy link

SparkQA commented Oct 23, 2014

QA tests have started for PR 2882 at commit 9514dc8.

  • This patch merges cleanly.

@tdas
Copy link
Contributor Author

tdas commented Oct 23, 2014

@JoshRosen
@harishreedharan addressed all your comments, and also simplified the writer code
I did some further cleanups, and also added two new unit tests that test the writer and manager with corrupted writes.

@SparkQA
Copy link

SparkQA commented Oct 23, 2014

QA tests have finished for PR 2882 at commit 9514dc8.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22068/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Oct 23, 2014

QA tests have started for PR 2882 at commit d29fddd.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 23, 2014

QA tests have finished for PR 2882 at commit d29fddd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22075/
Test PASSed.

@harishreedharan
Copy link
Contributor

Yay, finally!

@tdas
Copy link
Contributor Author

tdas commented Oct 23, 2014

@JoshRosen whenever you get a chance. :)

private[streaming] object HdfsUtils {

def getOutputStream(path: String, conf: Configuration): FSDataOutputStream = {
// HDFS is not thread-safe when getFileSystem is called, so synchronize on that
Copy link
Contributor

Choose a reason for hiding this comment

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

It looks like this comment is no longer relevant, or perhaps like it should be moved somewhere else?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Moved.

@JoshRosen
Copy link
Contributor

This looks good to me!

@tdas
Copy link
Contributor Author

tdas commented Oct 24, 2014

Alright, thanks! I will merge when this last set of changes gets through jenkins.

@SparkQA
Copy link

SparkQA commented Oct 24, 2014

QA tests have started for PR 2882 at commit e4bee20.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Oct 24, 2014

QA tests have finished for PR 2882 at commit e4bee20.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • case class LogInfo(startTime: Long, endTime: Long, path: String)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22103/
Test PASSed.

@harishreedharan
Copy link
Contributor

Let's merge this for now. I will try and find out more about the getFileSystem thread-safety without doAs (which is what we support anyway)

@harishreedharan
Copy link
Contributor

Talked to @cmccabe who says we should not worry about the thread-safety. If at all there was an issue, it was in too old a version which we need not worry about. Let's merge this!

@tdas
Copy link
Contributor Author

tdas commented Oct 24, 2014

Cool! Thanks for check with @cmccabe. Merging this.

@asfgit asfgit closed this in 6a40a76 Oct 24, 2014
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants