Skip to content

Commit

Permalink
[SPARK-23207][SPARK-22905][SPARK-24564][SPARK-25114][SQL][BACKPORT-2.…
Browse files Browse the repository at this point in the history
…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
jiangxb1987 authored and gatorsmile committed Aug 27, 2018
1 parent 09f70f5 commit 4d2d3d4
Show file tree
Hide file tree
Showing 20 changed files with 576 additions and 30 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ public abstract class RecordComparator {
public abstract int compare(
Object leftBaseObject,
long leftBaseOffset,
int leftBaseLength,
Object rightBaseObject,
long rightBaseOffset);
long rightBaseOffset,
int rightBaseLength);
}
Original file line number Diff line number Diff line change
Expand Up @@ -62,12 +62,13 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) {
int uaoSize = UnsafeAlignedOffset.getUaoSize();
if (prefixComparisonResult == 0) {
final Object baseObject1 = memoryManager.getPage(r1.recordPointer);
// skip length
final long baseOffset1 = memoryManager.getOffsetInPage(r1.recordPointer) + uaoSize;
final int baseLength1 = UnsafeAlignedOffset.getSize(baseObject1, baseOffset1 - uaoSize);
final Object baseObject2 = memoryManager.getPage(r2.recordPointer);
// skip length
final long baseOffset2 = memoryManager.getOffsetInPage(r2.recordPointer) + uaoSize;
return recordComparator.compare(baseObject1, baseOffset1, baseObject2, baseOffset2);
final int baseLength2 = UnsafeAlignedOffset.getSize(baseObject2, baseOffset2 - uaoSize);
return recordComparator.compare(baseObject1, baseOffset1, baseLength1, baseObject2,
baseOffset2, baseLength2);
} else {
return prefixComparisonResult;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,8 @@ public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) {
prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix());
if (prefixComparisonResult == 0) {
return recordComparator.compare(
left.getBaseObject(), left.getBaseOffset(),
right.getBaseObject(), right.getBaseOffset());
left.getBaseObject(), left.getBaseOffset(), left.getRecordLength(),
right.getBaseObject(), right.getBaseOffset(), right.getRecordLength());
} else {
return prefixComparisonResult;
}
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,8 @@ abstract class RDD[T: ClassTag](
*
* If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
* which can avoid performing a shuffle.
*
* TODO Fix the Shuffle+Repartition data loss issue described in SPARK-23207.
*/
def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
coalesce(numPartitions, shuffle = true)
Expand Down
10 changes: 10 additions & 0 deletions core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,10 @@

package org.apache.spark.memory;

import com.google.common.annotations.VisibleForTesting;

import org.apache.spark.unsafe.memory.MemoryBlock;

import java.io.IOException;

public class TestMemoryConsumer extends MemoryConsumer {
Expand All @@ -43,6 +47,12 @@ void free(long size) {
used -= size;
taskMemoryManager.releaseExecutionMemory(size, this);
}

@VisibleForTesting
public void freePage(MemoryBlock page) {
used -= page.size();
taskMemoryManager.freePage(page, this);
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -73,8 +73,10 @@ public class UnsafeExternalSorterSuite {
public int compare(
Object leftBaseObject,
long leftBaseOffset,
int leftBaseLength,
Object rightBaseObject,
long rightBaseOffset) {
long rightBaseOffset,
int rightBaseLength) {
return 0;
}
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,8 +97,10 @@ public void testSortingOnlyByIntegerPrefix() throws Exception {
public int compare(
Object leftBaseObject,
long leftBaseOffset,
int leftBaseLength,
Object rightBaseObject,
long rightBaseOffset) {
long rightBaseOffset,
int rightBaseLength) {
return 0;
}
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] {
val dataArray = Array.tabulate(weights.length) { i =>
Data(weights(i), gaussians(i).mu, gaussians(i).sigma)
}
spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path))
spark.createDataFrame(sc.makeRDD(dataArray, 1)).write.parquet(Loader.dataPath(path))
}

def load(sc: SparkContext, path: String): GaussianMixtureModel = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] {
val dataArray = Array.tabulate(model.selectedFeatures.length) { i =>
Data(model.selectedFeatures(i))
}
spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path))
spark.createDataFrame(sc.makeRDD(dataArray, 1)).write.parquet(Loader.dataPath(path))
}

def load(sc: SparkContext, path: String): ChiSqSelectorModel = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul
val oldModel = new OldWord2VecModel(word2VecMap)
val instance = new Word2VecModel("myWord2VecModel", oldModel)
val newInstance = testDefaultReadWrite(instance)
assert(newInstance.getVectors.collect() === instance.getVectors.collect())
assert(newInstance.getVectors.collect().sortBy(_.getString(0)) ===
instance.getVectors.collect().sortBy(_.getString(0)))
}

test("Word2Vec works with input that is non-nullable (NGram)") {
Expand Down
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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,26 +51,55 @@ public final class UnsafeExternalRowSorter {
private final PrefixComputer prefixComputer;
private final UnsafeExternalSorter sorter;

public static interface RecordComparatorSupplier {
public RecordComparator get();
}

public abstract static class PrefixComputer {

public static class Prefix {
/** Key prefix value, or the null prefix value if isNull = true. **/
long value;
public long value;

/** Whether the key is null. */
boolean isNull;
public boolean isNull;
}

/**
* Computes prefix for the given row. For efficiency, the returned object may be reused in
* further calls to a given PrefixComputer.
*/
abstract Prefix computePrefix(InternalRow row);
public abstract Prefix computePrefix(InternalRow row);
}

public static UnsafeExternalRowSorter createWithRecordComparator(
StructType schema,
RecordComparatorSupplier recordComparatorSupplier,
PrefixComparator prefixComparator,
PrefixComputer prefixComputer,
long pageSizeBytes,
boolean canUseRadixSort) throws IOException {
return new UnsafeExternalRowSorter(schema, recordComparatorSupplier, prefixComparator,
prefixComputer, pageSizeBytes, canUseRadixSort);
}

public static UnsafeExternalRowSorter create(
final StructType schema,
final Ordering<InternalRow> ordering,
PrefixComparator prefixComparator,
PrefixComputer prefixComputer,
long pageSizeBytes,
boolean canUseRadixSort) throws IOException {
RecordComparatorSupplier recordComparatorSupplier = new RecordComparatorSupplier() {
public RecordComparator get() { return new RowComparator(ordering, schema.length()); }
};
return new UnsafeExternalRowSorter(schema, recordComparatorSupplier, prefixComparator,
prefixComputer, pageSizeBytes, canUseRadixSort);
}

public UnsafeExternalRowSorter(
private UnsafeExternalRowSorter(
StructType schema,
Ordering<InternalRow> ordering,
RecordComparatorSupplier recordComparatorSupplier,
PrefixComparator prefixComparator,
PrefixComputer prefixComputer,
long pageSizeBytes,
Expand All @@ -84,7 +113,7 @@ public UnsafeExternalRowSorter(
sparkEnv.blockManager(),
sparkEnv.serializerManager(),
taskContext,
new RowComparator(ordering, schema.length()),
recordComparatorSupplier.get(),
prefixComparator,
sparkEnv.conf().getInt("spark.shuffle.sort.initialBufferSize",
DEFAULT_INITIAL_SORT_BUFFER_SIZE),
Expand Down Expand Up @@ -207,8 +236,15 @@ private static final class RowComparator extends RecordComparator {
}

@Override
public int compare(Object baseObj1, long baseOff1, Object baseObj2, long baseOff2) {
// TODO: Why are the sizes -1?
public int compare(
Object baseObj1,
long baseOff1,
int baseLen1,
Object baseObj2,
long baseOff2,
int baseLen2) {
// Note that since ordering doesn't need the total length of the record, we just pass -1
// into the row.
row1.pointTo(baseObj1, baseOff1, -1);
row2.pointTo(baseObj2, baseOff2, -1);
return ordering.compare(row1, row2);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -641,6 +641,18 @@ object SQLConf {
.booleanConf
.createWithDefault(false)

val SORT_BEFORE_REPARTITION =
SQLConfigBuilder("spark.sql.execution.sortBeforeRepartition")
.internal()
.doc("When perform a repartition following a shuffle, the output row ordering would be " +
"nondeterministic. If some downstream stages fail and some tasks of the repartition " +
"stage retry, these tasks may generate different data, and that can lead to correctness " +
"issues. Turn on this config to insert a local sort before actually doing repartition " +
"to generate consistent repartition results. The performance of repartition() may go " +
"down since we insert extra local sort before it.")
.booleanConf
.createWithDefault(true)

object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
Expand Down Expand Up @@ -758,6 +770,8 @@ class SQLConf extends Serializable with Logging {

def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)

def sortBeforeRepartition: Boolean = getConf(SORT_BEFORE_REPARTITION)

/**
* Returns the [[Resolver]] for the current configuration, which can be used to determine if two
* identifiers are equal.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -235,8 +235,14 @@ private static final class KVComparator extends RecordComparator {
}

@Override
public int compare(Object baseObj1, long baseOff1, Object baseObj2, long baseOff2) {
// Note that since ordering doesn't need the total length of the record, we just pass -1
public int compare(
Object baseObj1,
long baseOff1,
int baseLen1,
Object baseObj2,
long baseOff2,
int baseLen2) {
// Note that since ordering doesn't need the total length of the record, we just pass -1
// into the row.
row1.pointTo(baseObj1, baseOff1 + 4, -1);
row2.pointTo(baseObj2, baseOff2 + 4, -1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ case class SortExec(
}

val pageSize = SparkEnv.get.memoryManager.pageSizeBytes
val sorter = new UnsafeExternalRowSorter(
val sorter = UnsafeExternalRowSorter.create(
schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort)

if (testSpillFrequency > 0) {
Expand Down
Loading

0 comments on commit 4d2d3d4

Please sign in to comment.