-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-29435][Core]MapId in Shuffle Block is inconsistent at the writer and reader part when spark.shuffle.useOldFetchProtocol=true #26095
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 all commits
3d07479
939887d
d50744c
1e22dc3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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), | ||
|
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 the shuffle read side and we need to know the value of
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. This is redundant code, since ShuffleWrite writes the mapId based on the |
||
| serializerManager.wrapStream, | ||
| // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility | ||
| SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark | ||
|
|
||
| import org.scalatest.BeforeAndAfterAll | ||
|
|
||
| class ShuffleOldFetchProtocolSuite extends ShuffleSuite with BeforeAndAfterAll { | ||
|
|
||
| // This test suite should run all tests by setting spark.shuffle.useOldFetchProtocol=true. | ||
| override def beforeAll(): Unit = { | ||
| super.beforeAll() | ||
| conf.set("spark.shuffle.useOldFetchProtocol", "true") | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.
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.
Thanks for the report and fix!
The root cause is while we set
useOldFetchProtocol=truehere, 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 protocolOpenBlocks, 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:spark/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java
Line 296 in 148cd26
So the right way I think is doing the fix in
ShuffleWriteProcessor, we should fill mapId withmapTaskIdormapIndexdenpending on configspark.shuffle.useOldFetchProtocol.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.
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.