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-11419][STREAMING] Parallel recovery for FileBasedWriteAheadLog + minor recovery tweaks #9373

Closed
wants to merge 21 commits into from

Conversation

brkyvz
Copy link
Contributor

@brkyvz brkyvz commented Oct 30, 2015

The support for closing WriteAheadLog files after writes was just merged in. Closing every file after a write is a very expensive operation as it creates many small files on S3. It's not necessary to enable it on HDFS anyway.

However, when you have many small files on S3, recovery takes very long. In addition, files start stacking up pretty quickly, and deletes may not be able to keep up, therefore deletes can also be parallelized.

This PR adds support for the two parallelization steps mentioned above, in addition to a couple more failures I encountered during recovery.

@SparkQA
Copy link

SparkQA commented Oct 30, 2015

Test build #44665 has finished for PR 9373 at commit be5a2ab.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@brkyvz brkyvz changed the title [SPARK-11419][STREAMING] Par recovery [SPARK-11419][STREAMING] Parallel recovery for FileBasedWriteAheadLog + minor recovery tweaks Oct 30, 2015
receiverTracker.stop(processAllReceivedData)
if (receiverTracker != null) {
// First, stop receiving
receiverTracker.stop(processAllReceivedData)
Copy link
Contributor Author

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

@SparkQA
Copy link

SparkQA commented Oct 30, 2015

Test build #44666 has finished for PR 9373 at commit 7f8cfe3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -17,6 +17,7 @@
package org.apache.spark.streaming.util

import java.nio.ByteBuffer
import java.util.concurrent.ConcurrentSkipListSet
Copy link
Contributor Author

Choose a reason for hiding this comment

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

NoteToSelf: Remove

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 1, 2015

@harishreedharan Here are some benchmark results:
For reference, the driver was a r3.2xlarge EC2 instance.

image

Num Threads Rate (ms / file) Speed-up
50 5.556101934 9.004997951
25 5.99898194 8.340196225
8 8.692144733 5.756080699
4 14.1162362 3.544336169
1 50.03268653 1

@harishreedharan
Copy link
Contributor

Did you try HDFS? I am assuming we'd get similar speed ups there too but in
that case there are far fewer files in which case the cost to setup the
streams are paid only a handful of times.

What I am wondering is if we'd actually ever have to deal with that many
files in the non-S3 case. This adds the additional cost for HDFS or any
other FS, no? In those cases the number of files usually would be pretty
small, which may result in this being more expensive.

If this adds only a small cost or if it becomes faster, then let's keep
this.

On Sunday, November 1, 2015, Burak Yavuz <notifications@github.com
javascript:_e(%7B%7D,'cvml','notifications@github.com');> wrote:

@harishreedharan https://github.com/harishreedharan Here are some
benchmark results:
For reference, the driver was a r3.2xlarge EC2 instance.

[image: image]
https://cloud.githubusercontent.com/assets/5243515/10871515/54c14846-809e-11e5-91e6-2ac3605d98b7.png
Num Threads Rate (ms / file) Speed-up 50 5.556101934 9.004997951 25
5.99898194 8.340196225 8 8.692144733 5.756080699 4 14.1162362 3.544336169
1 50.03268653 1


Reply to this email directly or view it on GitHub
#9373 (comment).

Thanks,
Hari

Await.ready(f, 1 second)
oldLogFiles.foreach { logInfo =>
if (!executionContext.isShutdown) {
val f = Future { deleteFile(logInfo) }
Copy link
Contributor

Choose a reason for hiding this comment

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

Again should not use the default execution context. please make a execution context for this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

the execution context was defined implicitly in the class definition. Made it non-implicit for better readability

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 8, 2015

@harishreedharan I've been trying to test this patch, but I just couldn't set up HDFS to work with Spark using the spark-ec2 scripts. Could you please help me set up a cluster with HDFS so that I can benchmark this?
Basically, I can get HDFS up and running on the cluster, but Spark can't access it. get the following exception when I use Hadoop 2:

scala> sc.parallelize(1 to 5).saveAsTextFile("hdfs:///trial")
java.io.IOException: Failed on local exception: com.google.protobuf.InvalidProtocolBufferException: Protocol message contained an invalid tag (zero).; Host Details : local host is: "ip-172-31-13-113.us-west-2.compute.internal/172.31.13.113"; destination host is: "ec2-52-32-160-227.us-west-2.compute.amazonaws.com":9000;

That looks like a Protobuf version incompatibility.

I launched the ec2 instances using:

ec2/spark-ec2 --spark-git-repo=https://github.com/brkyvz/spark --spark-version=fc2951f6530bde932a0bc97f430c6c360eb03209 -s 2 --spot-price=0.2 -t m4.large --no-ganglia -i ... -k ... -r us-west-2 --hadoop-major-version 2 launch burak-streaming-test-2

I used to get the following when using --hadoop-major-version 1:

java.io.IOException: Failed on local exception: java.io.EOFException; Host Details : local host is: "ip-172-31-30-170.us-west-2.compute.internal/172.31.30.170"; destination host is: "ec2-52-32-211-30.us-west-2.compute.amazonaws.com":9000; 

@tdas
Copy link
Contributor

tdas commented Nov 9, 2015

@brkyvz I think there has been issues with Hadoop 2 related stuff in the master branch. Lets talk offline on how to fix it.

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 10, 2015

@harishreedharan I couldn't test this on HDFS properly. Instead I enabled the parallelization only when closeFileAfterWrite is enabled, which is when you actually really need it. Does that sound okay to you?

@tdas
Copy link
Contributor

tdas commented Nov 10, 2015

@brkyvz Could you update this PR with master? The batching PR got merged, creating conflicts.

@harishreedharan
Copy link
Contributor

@brkyvz Sounds good, sir. I think the issue you saw seems to be a protobuf incompatibility issue - did you compile and run against the same hadoop-2 version (2.2+ ?)
This patch now LGTM.

@SparkQA
Copy link

SparkQA commented Nov 10, 2015

Test build #45457 has finished for PR 9373 at commit 0b7279f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 10, 2015

Test build #45467 has finished for PR 9373 at commit c2cafe1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 10, 2015

Test build #45456 has finished for PR 9373 at commit 98da092.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@brkyvz
Copy link
Contributor Author

brkyvz commented Nov 11, 2015

test this please

@SparkQA
Copy link

SparkQA commented Nov 11, 2015

Test build #45650 has finished for PR 9373 at commit 1ba8340.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 11, 2015

Test build #45648 has finished for PR 9373 at commit 1ba8340.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

def getMax(): Int = synchronized { max }
}
try {
val testSeq = 1 to 64
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you make this 1000 instead of 8 * 8. Just to make sure that we are splitting things right.

@SparkQA
Copy link

SparkQA commented Nov 11, 2015

Test build #45668 has finished for PR 9373 at commit ccf7f5b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -582,6 +620,9 @@ object WriteAheadLogSuite {
allowBatching: Boolean): Seq[String] = {
val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching)
val data = wal.readAll().asScala.map(byteBufferToString).toSeq
// The thread pool for parallel recovery gets killed with wal.close(). Therefore we need to
// eagerly compute data, otherwise the lazy computation will fail.
data.length
Copy link
Member

Choose a reason for hiding this comment

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

Could you just change toSeq to toArray? toArray will drain the Iterator at once.

case ex: Exception =>
logWarning(s"Error clearing write ahead log file $logInfo", ex)
}
def deleteFile(walInfo: LogInfo): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: why rename this to walInfo?

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: empty line missing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

logInfo is Spark's logging method

@tdas
Copy link
Contributor

tdas commented Nov 12, 2015

@brkyvz Few more comments, and one pending comment from before about adding more unit tests.
@zsxwing please take a look once again.

@SparkQA
Copy link

SparkQA commented Nov 12, 2015

Test build #45700 has finished for PR 9373 at commit 7e1829b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 12, 2015

Test build #45712 has finished for PR 9373 at commit dbb31e3.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 12, 2015

Test build #45747 has finished for PR 9373 at commit a31822c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 12, 2015

Test build #45781 has finished for PR 9373 at commit 79e9b03.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@tdas
Copy link
Contributor

tdas commented Nov 13, 2015

LGTM. Merging this to master and 1.6. Thanks @brkyvz, @zsxwing and @harishreedharan

asfgit pushed a commit that referenced this pull request Nov 13, 2015
… + minor recovery tweaks

The support for closing WriteAheadLog files after writes was just merged in. Closing every file after a write is a very expensive operation as it creates many small files on S3. It's not necessary to enable it on HDFS anyway.

However, when you have many small files on S3, recovery takes very long. In addition, files start stacking up pretty quickly, and deletes may not be able to keep up, therefore deletes can also be parallelized.

This PR adds support for the two parallelization steps mentioned above, in addition to a couple more failures I encountered during recovery.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9373 from brkyvz/par-recovery.

(cherry picked from commit 7786f9c)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
@asfgit asfgit closed this in 7786f9c Nov 13, 2015
dskrvk pushed a commit to dskrvk/spark that referenced this pull request Nov 13, 2015
… + minor recovery tweaks

The support for closing WriteAheadLog files after writes was just merged in. Closing every file after a write is a very expensive operation as it creates many small files on S3. It's not necessary to enable it on HDFS anyway.

However, when you have many small files on S3, recovery takes very long. In addition, files start stacking up pretty quickly, and deletes may not be able to keep up, therefore deletes can also be parallelized.

This PR adds support for the two parallelization steps mentioned above, in addition to a couple more failures I encountered during recovery.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes apache#9373 from brkyvz/par-recovery.
@brkyvz brkyvz deleted the par-recovery branch February 3, 2019 20:59
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.

5 participants