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

[WIP][SPARK-35275][CORE] Add checksum for shuffle blocks and diagnose corruption #32385

Closed
wants to merge 9 commits into from

Conversation

Ngone51
Copy link
Member

@Ngone51 Ngone51 commented Apr 28, 2021

What changes were proposed in this pull request?

This PR proposes to add checksum support for shuffle blocks. The basic idea is:

On the mapper side, we'll wrap a CheckedOutputStream upon the FileOutputStream to calculate the checksum (use the same checksum calculator Adler32 with broadcast) for each shuffle block (a.k.a partition) at the same time when we writing map output files. And similar to the index file, we'll have a checksum file to save these checksums.

On the reducer side, we'll also wrap a CheckedInputStream upon the FileInputStream to read the block. When block corruption is detected, we'll try to diagnose corruption for the cause:

First, we'll use the CheckedInputStream to consume the remaining data of the corrupted block to calculate the checksum (c1);

Second, the reducer send an RPC request called DiagnoseCorruption (which contains c1) to the server (where the reducer executed)

Third, the server will read (using a very small memory) the corresponding block back from the disk and calculate the checksum (c2) again for it. And also read back the checksum(c3) of the block saved in the checksum file. Then, if c2 != c3, we'll suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we'll suspect the corruption is caused by the network issue. Otherwise, the cause remains unknown. The server then will reply to the reducer with CorruptionCause containing the cause.

Fourth, the reducer needs to take action after it receives the cause. If it's a disk issue or unknown, it will throw fetch failure directly. If it's a network issue, it will re-fetch the block later. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Overall, I think we don't introduce severe overhead with this proposal. In a normal case, the checksum is calculated in the streaming way as well as other streams, e.g., encryption, compression. And the major overhead here is that we need an extra data file traverse in the error case in order to calculate the checksum (c2).

And the proposal in this PR is much simpler compared to the previous one #15894 (abandoned due to complexity ), which introduce more overhead as it need to traverse the data file twice for every block. In that proposal, the checksum is appended to each block data, so it's also invasive to the existing code.

Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the checksum support for the shuffle, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

Does this PR introduce any user-facing change?

Yes.

  1. Added a conf spark.shuffle.checksum to let user enables/disables the checksum (enabled by default)
  2. With checksum enabled, users can know the possible cause of corruption rather than "Stream is corrupted" only.

How was this patch tested?

Added an end-to-end unit test in ShuffleSuite.

I'll add more tests if the community accepts the proposal.

@github-actions github-actions bot added the CORE label Apr 28, 2021
@Ngone51
Copy link
Member Author

Ngone51 commented Apr 28, 2021

I marked PR as WIP because I want to hear the community's feedback before working further, e.g., adding more unit tests.
And there will be two following PRs (if the community accept this proposal):

  1. Corruption diagnosis support for the batch fetched block

  2. Corruption diagnosis implementation for the external shuffle service

@Ngone51
Copy link
Member Author

Ngone51 commented Apr 28, 2021

cc @mridulm @otterc @attilapiros @tgravescs @cloud-fan Please take a look, thanks!

@SparkQA
Copy link

SparkQA commented Apr 28, 2021

Test build #138049 has finished for PR 32385 at commit c09ccd0.

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

@SparkQA
Copy link

SparkQA commented Apr 28, 2021

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42568/

@SparkQA
Copy link

SparkQA commented Apr 28, 2021

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42568/

@mridulm
Copy link
Contributor

mridulm commented Apr 28, 2021

+CC @otterc This should be of interest given recent discussions.

@otterc
Copy link
Contributor

otterc commented Apr 29, 2021

Thanks for copying me. I will take a look at it in few days.

@Ngone51 Ngone51 changed the title [WIP][SPARK-18188][CORE] Add checksum for shuffle blocks [WIP][SPARK-18188][CORE] Add checksum for shuffle blocks and diagnose corruption Apr 29, 2021
@Ngone51
Copy link
Member Author

Ngone51 commented Apr 29, 2021

Hi all, to ease the review for everyone, I have planned to split these PRs into 2 smaller PR first:

  1. calculate checksums and save them into a checksum file

(I have created the PR (#32401) for this part so you can start reviewing from there.)

  1. use the checksum file to diagnose the corruption.

(I'll create the PR later)

@Ngone51 Ngone51 changed the title [WIP][SPARK-18188][CORE] Add checksum for shuffle blocks and diagnose corruption [WIP][SPARK-35275][CORE] Add checksum for shuffle blocks and diagnose corruption Apr 29, 2021
@tgravescs
Copy link
Contributor

I haven't had time to look in detail, if this is a working prototype, have you done any performance measurements to see example impact this has? I know you said it should be minimal but some numbers would be nice.

Also at the high level how does this affect other shuffle work going on - like merging and pluggable? Is it independent of that or would need to be reimplemented?

@Ngone51
Copy link
Member Author

Ngone51 commented May 19, 2021

@tgravescs Thanks for the good points!

I did find some perf regression by benchmarking with the change. I'll double-check it for sure and try to get rid of it if possible.

Also at the high level how does this affect other shuffle work going on - like merging and pluggable? Is it independent of that or would need to be implemented?

For merging, it needs extension to send checksum values along with the block data while merging. The extension is also needed for the decommission feature.

For pluggable, my current implementation is added at LocalDiskShuffleMapOutputWriter, which is supposed to be the default shuffle writer plugin for Spark. It means, in this way, other custom plugins needs its own implementation for checksum support. I adopted that way becase I realized it's easier and more clear to implement at that time.

An alternative way to support checksum for all plugins or say to make it a built-in feature maybe is to implement it in DiskBlockObjectWriter/ShufflePartitionPairsWriter, which is the upstream to the shuffle I/O plugin. I need more investigation on this.

@Ngone51
Copy link
Member Author

Ngone51 commented Jun 8, 2021

Hi @tgravescs @mridulm @otterc , I have resolved the regression issue (verified by running the TPCDS benchmark with 3tb data internally) and made the checksum as a built-in feature of Spark.

And I have updated PR #32401 (which adds checksum support at shuffle writer side only) and I think it's ready for review.

@tgravescs
Copy link
Contributor

thanks @Ngone51, I'm very busy this week, so will take a look early next week.

@Ngone51
Copy link
Member Author

Ngone51 commented Jun 9, 2021

Sure, take your time :)

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Took one pass through the PR, thanks for working on this @Ngone51 !

if (!appId.equals(that.appId)) return false;
if (!execId.equals(that.execId)) return false;
if (!blockId.equals(that.blockId)) return false;
return checksum == that.checksum;
Copy link
Contributor

Choose a reason for hiding this comment

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

super nit: check checksum first ? cheapest check ..


@Override
public void encode(ByteBuf buf) {
buf.writeInt(cause.ordinal());
Copy link
Contributor

Choose a reason for hiding this comment

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

int -> byte ?

@@ -47,6 +48,15 @@
protected volatile TransportClientFactory clientFactory;
protected String appId;

public Cause diagnoseCorruption(
Copy link
Contributor

Choose a reason for hiding this comment

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

Include javadoc ?

int result = appId.hashCode();
result = 31 * result + execId.hashCode();
result = 31 * result + blockId.hashCode();
result = 31 * result + (int) checksum;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: checksum -> Long.hashCode(checksum) ?

this.partitionLengths = new long[numPartitions];
this.partitionChecksums = checksumEnabled ? new long[numPartitions] : new long[0];
Copy link
Contributor

Choose a reason for hiding this comment

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

We are using null in MapOutputCommitMessage while empty array here when checksum is disabled.
Unify to a single idiom ? Given writeMetadataFileAndCommit is depending on empty array (based on how it is written up right now), thoughts on using long[0] ?

(Btw, use a constant EMPTY_LONG_ARRAY if deciding to using new long[0]

}
}
} finally {
logDebug(s"Shuffle index for mapId $mapId: ${lengths.mkString("[", ",", "]")}")
if (indexTmp.exists() && !indexTmp.delete()) {
logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}")
}
checksumTmpOpt.foreach { checksumTmp =>
if (checksumTmp.exists() && !checksumTmp.delete()) {
logError(s"Failed to delete temporary checksum file at ${checksumTmp.getAbsolutePath}")
Copy link
Contributor

Choose a reason for hiding this comment

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

logInfo ?

val channel = Files.newByteChannel(checksumFile.toPath)
channel.position(reduceId * 8L)
in = new DataInputStream(Channels.newInputStream(channel))
val goldenChecksum = in.readLong()
Copy link
Contributor

Choose a reason for hiding this comment

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

Extract out a readChecksum and computeChecksum methods ?
Btw, tryWithResource { DataInputStream(FileInputStream()).skip(reduceId * 8L).readLong() } would do the trick for readChecksum.

val checksumIn = new CheckedInputStream(blockData.createInputStream(), new Adler32)
val buffer = new Array[Byte](8192)
while (checksumIn.read(buffer, 0, 8192) != -1) {}
val recalculatedChecksum = checksumIn.getChecksum.getValue
Copy link
Contributor

Choose a reason for hiding this comment

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

We are not closing checksumIn btw.

@@ -76,6 +77,9 @@ private[spark] class DiskBlockObjectWriter(
private var initialized = false
private var streamOpen = false
private var hasBeenClosed = false
private var checksumEnabled = false
private var checksumCal: Checksum = null
private var checksumOutputStream: CheckedOutputStream = null
Copy link
Contributor

Choose a reason for hiding this comment

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

Same comment as above - reduce the number of streaming as fields ?


override def close(): Unit = sink.close()

}
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice unification !

@Ngone51
Copy link
Member Author

Ngone51 commented Jun 15, 2021

oh..@mridulm Sorry if I confused you here. I have planed to split this PR into two separate PRs to ease the review:

So please help review the smaller PR there.

And I'll try to resolve your comments in the separate PRs. Thanks!

@mridulm
Copy link
Contributor

mridulm commented Jun 16, 2021

lol, thanks for the links @Ngone51 :-)
Glad I went through this once more anyway - will help me with better understanding of the sub-pr's !
Will wait for the update before taking a look at #32401.

asfgit pushed a commit that referenced this pull request Jul 17, 2021
…checksum file

### What changes were proposed in this pull request?

This is the initial work of add checksum support of shuffle. This is a piece of #32385. And this PR only adds checksum functionality at the shuffle writer side.

Basically, the idea is to wrap a `MutableCheckedOutputStream`* upon the `FileOutputStream` while the shuffle writer generating the shuffle data. But the specific wrapping places are a bit different among the shuffle writers due to their different implementation:

* `BypassMergeSortShuffleWriter` -  wrap on each partition file
* `UnsafeShuffleWriter` - wrap on each spill files directly since they doesn't require aggregation, sorting
* `SortShuffleWriter` - wrap on the `ShufflePartitionPairsWriter` after merged spill files since they might require aggregation, sorting

\* `MutableCheckedOutputStream` is a variant of `java.util.zip.CheckedOutputStream` which can change the checksum calculator at runtime.

And we use the `Adler32`, which uses the CRC-32 algorithm but much faster, to calculate the checksum as the same as `Broadcast`'s checksum.

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?

Yes, added a new conf: `spark.shuffle.checksum`.

### How was this patch tested?

Added unit tests.

Closes #32401 from Ngone51/add-checksum-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
asfgit pushed a commit that referenced this pull request Jul 17, 2021
…checksum file

### What changes were proposed in this pull request?

This is the initial work of add checksum support of shuffle. This is a piece of #32385. And this PR only adds checksum functionality at the shuffle writer side.

Basically, the idea is to wrap a `MutableCheckedOutputStream`* upon the `FileOutputStream` while the shuffle writer generating the shuffle data. But the specific wrapping places are a bit different among the shuffle writers due to their different implementation:

* `BypassMergeSortShuffleWriter` -  wrap on each partition file
* `UnsafeShuffleWriter` - wrap on each spill files directly since they doesn't require aggregation, sorting
* `SortShuffleWriter` - wrap on the `ShufflePartitionPairsWriter` after merged spill files since they might require aggregation, sorting

\* `MutableCheckedOutputStream` is a variant of `java.util.zip.CheckedOutputStream` which can change the checksum calculator at runtime.

And we use the `Adler32`, which uses the CRC-32 algorithm but much faster, to calculate the checksum as the same as `Broadcast`'s checksum.

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?

Yes, added a new conf: `spark.shuffle.checksum`.

### How was this patch tested?

Added unit tests.

Closes #32401 from Ngone51/add-checksum-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 4783fb7)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
asfgit pushed a commit that referenced this pull request Aug 2, 2021
…ffle checksum

### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out #32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

### Does this PR introduce _any_ user-facing change?

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes #33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
asfgit pushed a commit that referenced this pull request Aug 2, 2021
…ffle checksum

### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out #32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

### Does this PR introduce _any_ user-facing change?

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes #33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
@github-actions
Copy link

We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!

@github-actions github-actions bot added the Stale label Sep 25, 2021
@github-actions github-actions bot closed this Sep 26, 2021
venkata91 pushed a commit to linkedin/spark that referenced this pull request Oct 13, 2021
…checksum file

This is the initial work of add checksum support of shuffle. This is a piece of apache#32385. And this PR only adds checksum functionality at the shuffle writer side.

Basically, the idea is to wrap a `MutableCheckedOutputStream`* upon the `FileOutputStream` while the shuffle writer generating the shuffle data. But the specific wrapping places are a bit different among the shuffle writers due to their different implementation:

* `BypassMergeSortShuffleWriter` -  wrap on each partition file
* `UnsafeShuffleWriter` - wrap on each spill files directly since they doesn't require aggregation, sorting
* `SortShuffleWriter` - wrap on the `ShufflePartitionPairsWriter` after merged spill files since they might require aggregation, sorting

\* `MutableCheckedOutputStream` is a variant of `java.util.zip.CheckedOutputStream` which can change the checksum calculator at runtime.

And we use the `Adler32`, which uses the CRC-32 algorithm but much faster, to calculate the checksum as the same as `Broadcast`'s checksum.

Yes, added a new conf: `spark.shuffle.checksum`.

Added unit tests.

Closes apache#32401 from Ngone51/add-checksum-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 4783fb7)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
venkata91 pushed a commit to linkedin/spark that referenced this pull request Oct 13, 2021
…ffle checksum

### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out apache#32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

### Does this PR introduce _any_ user-facing change?

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes apache#33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
@pan3793
Copy link
Member

pan3793 commented Feb 27, 2022

Hi @Ngone51, thanks for providing the checksum feature for shuffle, have a thought about the following case.

if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs.

I think the checksum could handle this case. Pick the idea from #28525, we can consume the input stream inside ShuffleBlockFetcherIterator and verify the checksum immediately, then

If it's a disk issue or unknown, it will throw fetch failure directly. If it's a network issue, it will re-fetch the block later.

To achieve this, we may need to update the shuffle network protocol to support passing checksums when fetching shuffle blocks.

Compare to the existing Utils.copyStreamUpTo(input, maxBytesInFlight / 3), this approach use less memory and can verify any size of blocks, but it introduce another overhead because it need to read the data 2 times.

We encounter this issues in our production everyday, for some jobs, the performance overhead is acceptable comparing to stability.

@Ngone51 WDYT?

domybest11 pushed a commit to domybest11/spark that referenced this pull request Jun 15, 2022
…checksum file

### What changes were proposed in this pull request?

This is the initial work of add checksum support of shuffle. This is a piece of apache#32385. And this PR only adds checksum functionality at the shuffle writer side.

Basically, the idea is to wrap a `MutableCheckedOutputStream`* upon the `FileOutputStream` while the shuffle writer generating the shuffle data. But the specific wrapping places are a bit different among the shuffle writers due to their different implementation:

* `BypassMergeSortShuffleWriter` -  wrap on each partition file
* `UnsafeShuffleWriter` - wrap on each spill files directly since they doesn't require aggregation, sorting
* `SortShuffleWriter` - wrap on the `ShufflePartitionPairsWriter` after merged spill files since they might require aggregation, sorting

\* `MutableCheckedOutputStream` is a variant of `java.util.zip.CheckedOutputStream` which can change the checksum calculator at runtime.

And we use the `Adler32`, which uses the CRC-32 algorithm but much faster, to calculate the checksum as the same as `Broadcast`'s checksum.

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?

Yes, added a new conf: `spark.shuffle.checksum`.

### How was this patch tested?

Added unit tests.

Closes apache#32401 from Ngone51/add-checksum-files.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit 4783fb7)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
domybest11 pushed a commit to domybest11/spark that referenced this pull request Jun 15, 2022
…ffle checksum

### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out apache#32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

### Does this PR introduce _any_ user-facing change?

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes apache#33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
domybest11 pushed a commit to domybest11/spark that referenced this pull request Jun 15, 2022
…ffle checksum

### What changes were proposed in this pull request?

This PR adds support to diagnose shuffle data corruption. Basically, the diagnosis mechanism works like this:
The shuffler reader would calculate the checksum (c1) for the corrupted shuffle block and send it to the server where the block is stored. At the server, it would read back the checksum (c2) that is stored in the checksum file and recalculate the checksum (c3) for the corresponding shuffle block. Then, if c2 != c3, we suspect the corruption is caused by the disk issue. Otherwise, if c1 != c3, we suspect the corruption is caused by the network issue. Otherwise, the checksum verifies pass. In any case of the error, the cause remains unknown.

After the shuffle reader receives the diagnosis response, it'd take the action bases on the type of cause. Only in case of the network issue, we'd give a retry. Otherwise, we'd throw the fetch failure directly. Also note that, if the corruption happens inside BufferReleasingInputStream, the reducer will throw the fetch failure immediately no matter what the cause is since the data has been partially consumed by downstream RDDs. If corruption happens again after retry, the reducer will throw the fetch failure directly this time without the diagnosis.

Please check out apache#32385 to see the completed proposal of the shuffle checksum project.

### Why are the changes needed?

Shuffle data corruption is a long-standing issue in Spark. For example, in SPARK-18105, people continually reports corruption issue. However, data corruption is difficult to reproduce in most cases and even harder to tell the root cause. We don't know if it's a Spark issue or not. With the diagnosis support for the shuffle corruption, Spark itself can at least distinguish the cause between disk and network, which is very important for users.

### Does this PR introduce _any_ user-facing change?

Yes, users may know the cause of the shuffle corruption after this change.

### How was this patch tested?

Added tests.

Closes apache#33451 from Ngone51/SPARK-36206.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
(cherry picked from commit a98d919)
Signed-off-by: Mridul Muralidharan <mridulatgmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
6 participants