-
Notifications
You must be signed in to change notification settings - Fork 28.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
…1] Shuffle+Repartition on a DataFrame could lead to incorrect answers ## What changes were proposed in this pull request? Back port of #20393 and #22079. Currently shuffle repartition uses RoundRobinPartitioning, the generated result is nondeterministic since the sequence of input rows are not determined. The bug can be triggered when there is a repartition call following a shuffle (which would lead to non-deterministic row ordering), as the pattern shows below: upstream stage -> repartition stage -> result stage (-> indicate a shuffle) When one of the executors process goes down, some tasks on the repartition stage will be retried and generate inconsistent ordering, and some tasks of the result stage will be retried generating different data. The following code returns 931532, instead of 1000000: ``` import scala.sys.process._ import org.apache.spark.TaskContext val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x => x }.repartition(200).map { x => if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) { throw new Exception("pkill -f java".!!) } x } res.distinct().count() ``` In this PR, we propose a most straight-forward way to fix this problem by performing a local sort before partitioning, after we make the input row ordering deterministic, the function from rows to partitions is fully deterministic too. The downside of the approach is that with extra local sort inserted, the performance of repartition() will go down, so we add a new config named `spark.sql.execution.sortBeforeRepartition` to control whether this patch is applied. The patch is default enabled to be safe-by-default, but user may choose to manually turn it off to avoid performance regression. This patch also changes the output rows ordering of repartition(), that leads to a bunch of test cases failure because they are comparing the results directly. Add unit test in ExchangeSuite. With this patch(and `spark.sql.execution.sortBeforeRepartition` set to true), the following query returns 1000000: ``` import scala.sys.process._ import org.apache.spark.TaskContext spark.conf.set("spark.sql.execution.sortBeforeRepartition", "true") val res = spark.range(0, 1000 * 1000, 1).repartition(200).map { x => x }.repartition(200).map { x => if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId < 2) { throw new Exception("pkill -f java".!!) } x } res.distinct().count() res7: Long = 1000000 ``` Author: Xingbo Jiang <xingbo.jiangdatabricks.com> Author: Xingbo Jiang <[email protected]> Author: Henry Robinson <[email protected]> Closes #22211 from henryr/spark-23207-branch-2.1.
- Loading branch information
1 parent
09f70f5
commit 4d2d3d4
Showing
20 changed files
with
576 additions
and
30 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
74 changes: 74 additions & 0 deletions
74
sql/catalyst/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* 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.sql.execution; | ||
|
||
import org.apache.spark.unsafe.Platform; | ||
import org.apache.spark.util.collection.unsafe.sort.RecordComparator; | ||
|
||
public final class RecordBinaryComparator extends RecordComparator { | ||
|
||
@Override | ||
public int compare( | ||
Object leftObj, long leftOff, int leftLen, Object rightObj, long rightOff, int rightLen) { | ||
int i = 0; | ||
|
||
// If the arrays have different length, the longer one is larger. | ||
if (leftLen != rightLen) { | ||
return leftLen - rightLen; | ||
} | ||
|
||
// The following logic uses `leftLen` as the length for both `leftObj` and `rightObj`, since | ||
// we have guaranteed `leftLen` == `rightLen`. | ||
|
||
// check if stars align and we can get both offsets to be aligned | ||
if ((leftOff % 8) == (rightOff % 8)) { | ||
while ((leftOff + i) % 8 != 0 && i < leftLen) { | ||
final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; | ||
final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; | ||
if (v1 != v2) { | ||
return v1 > v2 ? 1 : -1; | ||
} | ||
i += 1; | ||
} | ||
} | ||
// for architectures that support unaligned accesses, chew it up 8 bytes at a time | ||
if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) { | ||
while (i <= leftLen - 8) { | ||
final long v1 = Platform.getLong(leftObj, leftOff + i); | ||
final long v2 = Platform.getLong(rightObj, rightOff + i); | ||
if (v1 != v2) { | ||
return v1 > v2 ? 1 : -1; | ||
} | ||
i += 8; | ||
} | ||
} | ||
// this will finish off the unaligned comparisons, or do the entire aligned comparison | ||
// whichever is needed. | ||
while (i < leftLen) { | ||
final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; | ||
final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; | ||
if (v1 != v2) { | ||
return v1 > v2 ? 1 : -1; | ||
} | ||
i += 1; | ||
} | ||
|
||
// The two arrays are equal. | ||
return 0; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.