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-29435][Core]MapId in Shuffle Block is inconsistent at the writer and reader part when spark.shuffle.useOldFetchProtocol=true #26095

Closed
wants to merge 4 commits into from

Conversation

sandeep-katta
Copy link
Contributor

@sandeep-katta sandeep-katta commented Oct 11, 2019

What changes were proposed in this pull request?

Shuffle Block Construction during Shuffle Write and Read is wrong

Shuffle Map Task (Shuffle Write)
19/10/11 22:07:32| ERROR| [Executor task launch worker for task 3] org.apache.spark.shuffle.IndexShuffleBlockResolver: ####### For Debug ############ /tmp/hadoop-root1/nm-local-dir/usercache/root1/appcache/application_1570422377362_0008/blockmgr-6d03250d-6e7c-4bc2-bbb7-22b8e3981c35/0d/shuffle_0_3_0.index

Result Task (Shuffle Read)
19/10/11 22:07:32| ERROR| [Executor task launch worker for task 6] org.apache.spark.storage.ShuffleBlockFetcherIterator: Error occurred while fetching local blocks
java.nio.file.NoSuchFileException: /tmp/hadoop-root1/nm-local-dir/usercache/root1/appcache/application_1570422377362_0008/blockmgr-6d03250d-6e7c-4bc2-bbb7-22b8e3981c35/30/shuffle_0_0_0.index
at sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)

As per SPARK-25341 mapId of SortShuffleManager.getWriter changed to context.taskAttemptId() from partitionId

code

But MapOutputTracker.convertMapStatuses returns the wrong ShuffleBlock, if spark.shuffle.useOldFetchProtocol enabled, it returns paritionId as mapID which is wrong . Code

Why are the changes needed?

Already MapStatus returned by the ShuffleWriter has the mapId for e.g. code here. So it's nice to use status.mapTaskId

Does this PR introduce any user-facing change?

No

How was this patch tested?

Existing UT and manually tested with spark.shuffle.useOldFetchProtocol as true and false

image

@sandeep-katta
Copy link
Contributor Author

cc @cloud-fan @xuanyuanking Please take a look at the fix, if it is okay I will add UT and refactor the code

@@ -905,15 +905,8 @@ private[spark] object MapOutputTracker extends Logging {
for (part <- startPartition until endPartition) {
val size = status.getSizeForBlock(part)
if (size != 0) {
if (useOldFetchProtocol) {
Copy link
Member

@xuanyuanking xuanyuanking Oct 13, 2019

Choose a reason for hiding this comment

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

Thanks for the report and fix!
The root cause is while we set useOldFetchProtocol=true here, the shuffle id in the reader side and the writer side are inconsistent.
But we can't fix like this, because while useOldFetchProtocl=false, we'll use the old version of fetching protocol OpenBlocks, which consider map id is Integer and will directly parse the string. So for the big and long-running application, it will still not work. See the code:

mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]);

So the right way I think is doing the fix in ShuffleWriteProcessor, we should fill mapId with mapTaskId or mapIndex denpending on config spark.shuffle.useOldFetchProtocol.

Copy link
Member

Choose a reason for hiding this comment

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

sorry, could you explain why Integer directly parse the string for the big and long-running application not work?
Is it a performance problem?
looking forward for your reply.

@@ -47,8 +47,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
context,
blockManager.blockStoreClient,
blockManager,
mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition,
SparkEnv.get.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)),
mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition),
Copy link
Contributor

Choose a reason for hiding this comment

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

This is the shuffle read side and we need to know the value of SHUFFLE_USE_OLD_FETCH_PROTOCOL. I think the bug is in the shuffle write side which is fixed in this PR. Do we really need to change the shuffle read side?

Copy link
Contributor Author

@sandeep-katta sandeep-katta Oct 14, 2019

Choose a reason for hiding this comment

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

This is redundant code, since ShuffleWrite writes the mapId based on the spark.shuffle.useOldFetchProtocol flag, MapStatus.mapTaskId always gives the mapId which is set by the ShuffleWriter

((ShuffleBlockId(shuffleId, status.mapTaskId, part), size, mapIndex))
}
splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) +=
((ShuffleBlockId(shuffleId, status.mapTaskId, part), size, mapIndex))
Copy link
Contributor

Choose a reason for hiding this comment

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

here we always pick status.mapTaskId as mapId, is this corrected?

Copy link
Contributor

Choose a reason for hiding this comment

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

OK I get it now. We should rename MapStatus.mapTaskId to mapId.

@cloud-fan
Copy link
Contributor

ok to test

@xuanyuanking
Copy link
Member

Please also change the PR description, the detailed log and code can be skipped, even the code linked with master will always in changing.
We need to mention the consistency between shuffle reader/writer while using old protocol, and the limitation of using mapTaskId in OpenBlock message.

@sandeep-katta sandeep-katta changed the title [SPARK-29435][Core]Shuffle is not working when spark.shuffle.useOldFetchProtocol=true [SPARK-29435][Core]MapId in Shuffle Block is inconsistent at the writer and reader part when spark.shuffle.useOldFetchProtocol=true Oct 14, 2019
@SparkQA
Copy link

SparkQA commented Oct 14, 2019

Test build #112010 has finished for PR 26095 at commit 939887d.

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

@xuanyuanking
Copy link
Member

@sandeep-katta In case of the consistency issue, we'd better add a UT for the old fetch protocol config. I gave a PR to your branch, please have a review: sandeep-katta#3.
In the current master the UT will fail, and in your branch it will pass.

@SparkQA
Copy link

SparkQA commented Oct 14, 2019

Test build #112014 has finished for PR 26095 at commit d50744c.

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

@SparkQA
Copy link

SparkQA commented Oct 14, 2019

Test build #112028 has finished for PR 26095 at commit 1e22dc3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class ShuffleOldFetchProtocolSuite extends ShuffleSuite with BeforeAndAfterAll

@cloud-fan
Copy link
Contributor

thanks, merging to master!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants