JiexingLi commented on code in PR #50230:
URL: https://github.com/apache/spark/pull/50230#discussion_r2050974242


##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.shuffle.checksum
+
+import java.io.{ByteArrayOutputStream, ObjectOutputStream}
+import java.util.zip.Checksum
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper
+
+/**
+ * A class for computing checksum for input (key, value) pairs. The checksum 
is independent of
+ * the order of the input (key, value) pairs. It is done by computing a 
checksum for each row
+ * first, and then computing the XOR for all the row checksums.
+ */
+abstract class RowBasedChecksum() extends Serializable with Logging {
+  private var hasError: Boolean = false
+  private var checksumValue: Long = 0
+  /** Returns the checksum value computed. Tt returns the default checksum 
value (0) if there
+   * are any errors encountered during the checksum computation.
+   */
+  def getValue: Long = {
+    if (!hasError) checksumValue else 0
+  }
+
+  /** Updates the row-based checksum with the given (key, value) pair */
+  def update(key: Any, value: Any): Unit = {
+    if (!hasError) {
+      try {
+        val rowChecksumValue = calculateRowChecksum(key, value)
+        checksumValue = checksumValue ^ rowChecksumValue
+      } catch {
+        case NonFatal(e) =>
+          logInfo("Checksum computation encountered error: ", e)
+          hasError = true
+      }
+    }
+  }
+
+  /** Computes and returns the checksum value for the given (key, value) pair 
*/
+  protected def calculateRowChecksum(key: Any, value: Any): Long
+}
+
+/**
+ * A Concrete implementation of RowBasedChecksum. The checksum for each row is
+ * computed by first converting the (key, value) pair to byte array using 
OutputStreams,
+ * and then computing the checksum for the byte array.
+ *
+ * @param checksumAlgorithm the algorithm used for computing checksum.
+ */
+class OutputStreamRowBasedChecksum(checksumAlgorithm: String)
+  extends RowBasedChecksum() {
+
+  /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */
+  final private class MyByteArrayOutputStream(size: Int)
+    extends ByteArrayOutputStream(size) {
+    def getBuf: Array[Byte] = buf
+  }
+
+  private val DEFAULT_INITIAL_SER_BUFFER_SIZE = 32 * 1024
+
+  @transient private lazy val serBuffer =
+    new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE)
+  @transient private lazy val objOut = new ObjectOutputStream(serBuffer)

Review Comment:
   OutputStreamRowBasedChecksum is actually only used in testing only. The one 
that will run in actual production is in UnsafeRowChecksum.scala, where we 
don't make any copy of the data. The overhead for UnsafeRowChecksum is around 
0.4% - 0.7%. I added to the test part of the PR description.
   
   I can't use UnsafeRowChecksum.scala in the test because the test is in core, 
while the usaferow is in sql. So I added OutputStreamRowBasedChecksum for the 
tests in core.



##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.shuffle.checksum
+
+import java.io.{ByteArrayOutputStream, ObjectOutputStream}
+import java.util.zip.Checksum
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper
+
+/**
+ * A class for computing checksum for input (key, value) pairs. The checksum 
is independent of
+ * the order of the input (key, value) pairs. It is done by computing a 
checksum for each row
+ * first, and then computing the XOR for all the row checksums.
+ */
+abstract class RowBasedChecksum() extends Serializable with Logging {
+  private var hasError: Boolean = false
+  private var checksumValue: Long = 0
+  /** Returns the checksum value computed. Tt returns the default checksum 
value (0) if there
+   * are any errors encountered during the checksum computation.
+   */
+  def getValue: Long = {
+    if (!hasError) checksumValue else 0
+  }
+
+  /** Updates the row-based checksum with the given (key, value) pair */
+  def update(key: Any, value: Any): Unit = {
+    if (!hasError) {
+      try {
+        val rowChecksumValue = calculateRowChecksum(key, value)
+        checksumValue = checksumValue ^ rowChecksumValue
+      } catch {
+        case NonFatal(e) =>
+          logInfo("Checksum computation encountered error: ", e)
+          hasError = true
+      }
+    }
+  }
+
+  /** Computes and returns the checksum value for the given (key, value) pair 
*/
+  protected def calculateRowChecksum(key: Any, value: Any): Long
+}
+
+/**
+ * A Concrete implementation of RowBasedChecksum. The checksum for each row is
+ * computed by first converting the (key, value) pair to byte array using 
OutputStreams,
+ * and then computing the checksum for the byte array.
+ *
+ * @param checksumAlgorithm the algorithm used for computing checksum.
+ */
+class OutputStreamRowBasedChecksum(checksumAlgorithm: String)
+  extends RowBasedChecksum() {
+
+  /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */
+  final private class MyByteArrayOutputStream(size: Int)
+    extends ByteArrayOutputStream(size) {
+    def getBuf: Array[Byte] = buf
+  }

Review Comment:
   Can you help pointing me to where is it? This class is only used in test.



##########
core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java:
##########
@@ -171,7 +182,11 @@ public void write(Iterator<Product2<K, V>> records) throws 
IOException {
       while (records.hasNext()) {
         final Product2<K, V> record = records.next();
         final K key = record._1();
-        partitionWriters[partitioner.getPartition(key)].write(key, 
record._2());
+        final int partitionId = partitioner.getPartition(key);
+        partitionWriters[partitionId].write(key, record._2());
+        if (rowBasedChecksums.length > 0) {

Review Comment:
   Updated to all use length/isEmpty now 



##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.shuffle.checksum
+
+import java.io.{ByteArrayOutputStream, ObjectOutputStream}
+import java.util.zip.Checksum
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper
+
+/**
+ * A class for computing checksum for input (key, value) pairs. The checksum 
is independent of
+ * the order of the input (key, value) pairs. It is done by computing a 
checksum for each row
+ * first, and then computing the XOR for all the row checksums.
+ */
+abstract class RowBasedChecksum() extends Serializable with Logging {
+  private var hasError: Boolean = false
+  private var checksumValue: Long = 0
+  /** Returns the checksum value computed. Tt returns the default checksum 
value (0) if there
+   * are any errors encountered during the checksum computation.
+   */
+  def getValue: Long = {
+    if (!hasError) checksumValue else 0
+  }
+
+  /** Updates the row-based checksum with the given (key, value) pair */
+  def update(key: Any, value: Any): Unit = {
+    if (!hasError) {
+      try {
+        val rowChecksumValue = calculateRowChecksum(key, value)
+        checksumValue = checksumValue ^ rowChecksumValue
+      } catch {
+        case NonFatal(e) =>
+          logInfo("Checksum computation encountered error: ", e)

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/Dependency.scala:
##########
@@ -83,9 +85,30 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: 
ClassTag](
     val keyOrdering: Option[Ordering[K]] = None,
     val aggregator: Option[Aggregator[K, V, C]] = None,
     val mapSideCombine: Boolean = false,
-    val shuffleWriterProcessor: ShuffleWriteProcessor = new 
ShuffleWriteProcessor)
+    val shuffleWriterProcessor: ShuffleWriteProcessor = new 
ShuffleWriteProcessor,
+    val rowBasedChecksums: Array[RowBasedChecksum] = Array.empty)

Review Comment:
   Done



##########
core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java:
##########
@@ -199,6 +214,15 @@ public long[] getPartitionLengths() {
     return partitionLengths;
   }
 
+  public RowBasedChecksum[] getRowBasedChecksums() {
+    return rowBasedChecksums;
+  }
+
+  public long getAggregatedChecksumValue() {
+    final long checksum = 
RowBasedChecksum.getAggregatedChecksumValue(rowBasedChecksums);
+    return checksum;

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/MapOutputTracker.scala:
##########
@@ -169,6 +174,12 @@ private class ShuffleStatus(
     } else {
       mapIdToMapIndex.remove(currentMapStatus.mapId)
     }
+
+    val preStatus =
+      if (mapStatuses(mapIndex) != null) mapStatuses(mapIndex) else 
mapStatusesDeleted(mapIndex)
+    if (preStatus != null && preStatus.checksumValue != status.checksumValue) {
+      checksumMismatchIndices.add(mapIndex)
+    }

Review Comment:
   Yes, for case 1, we need to track the mismatches. The usage of 
checksumMismatchIndices is that (in the next PR) we will rollback the 
downstream stages, if we detect checksum mismatches for its upstream stages.
   
   For case 2, if downstream stages have not consumed output, which means they 
have not started. In this case, the rollback is a no-op, and it doesn't hurt to 
record the mismatches here.
   
   For case 3, I think we need to record the mismatches. Assuming a situation 
where all partitions of a stage have finished, while some speculative tasks are 
still running. As all outputs have been produced, the downstream stage can 
start and read from the data. Later, some speculative tasks finish, and new 
mapStatus will override the old mapStatus with new data location. For the 
downstream stage, the not yet started tasks or retried tasks would read from 
the new data, while the finished and running tasks would read from the old 
data, resulting in inconsistency.



##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.shuffle.checksum
+
+import java.io.{ByteArrayOutputStream, ObjectOutputStream}
+import java.util.zip.Checksum
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper
+
+/**
+ * A class for computing checksum for input (key, value) pairs. The checksum 
is independent of
+ * the order of the input (key, value) pairs. It is done by computing a 
checksum for each row
+ * first, and then computing the XOR for all the row checksums.
+ */
+abstract class RowBasedChecksum() extends Serializable with Logging {
+  private var hasError: Boolean = false
+  private var checksumValue: Long = 0
+  /** Returns the checksum value computed. Tt returns the default checksum 
value (0) if there
+   * are any errors encountered during the checksum computation.
+   */
+  def getValue: Long = {
+    if (!hasError) checksumValue else 0
+  }
+
+  /** Updates the row-based checksum with the given (key, value) pair */
+  def update(key: Any, value: Any): Unit = {
+    if (!hasError) {
+      try {
+        val rowChecksumValue = calculateRowChecksum(key, value)
+        checksumValue = checksumValue ^ rowChecksumValue
+      } catch {
+        case NonFatal(e) =>
+          logInfo("Checksum computation encountered error: ", e)
+          hasError = true
+      }
+    }
+  }
+
+  /** Computes and returns the checksum value for the given (key, value) pair 
*/
+  protected def calculateRowChecksum(key: Any, value: Any): Long
+}
+
+/**
+ * A Concrete implementation of RowBasedChecksum. The checksum for each row is
+ * computed by first converting the (key, value) pair to byte array using 
OutputStreams,
+ * and then computing the checksum for the byte array.
+ *
+ * @param checksumAlgorithm the algorithm used for computing checksum.
+ */
+class OutputStreamRowBasedChecksum(checksumAlgorithm: String)
+  extends RowBasedChecksum() {
+
+  /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */
+  final private class MyByteArrayOutputStream(size: Int)
+    extends ByteArrayOutputStream(size) {
+    def getBuf: Array[Byte] = buf
+  }
+
+  private val DEFAULT_INITIAL_SER_BUFFER_SIZE = 32 * 1024
+
+  @transient private lazy val serBuffer =
+    new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE)
+  @transient private lazy val objOut = new ObjectOutputStream(serBuffer)
+
+  @transient
+  protected lazy val checksum: Checksum =
+    ShuffleChecksumHelper.getChecksumByAlgorithm(checksumAlgorithm)
+
+  override protected def calculateRowChecksum(key: Any, value: Any): Long = {
+    assert(checksum != null, "Checksum is null")
+
+    // Converts the (key, value) pair into byte array.
+    objOut.reset()
+    serBuffer.reset()
+    objOut.writeObject((key, value))
+    objOut.flush()
+    serBuffer.flush()
+
+    // Computes and returns the checksum for the byte array.
+    checksum.reset()
+    checksum.update(serBuffer.getBuf, 0, serBuffer.size())
+    checksum.getValue
+  }
+}
+
+object RowBasedChecksum {
+  def createPartitionRowBasedChecksums(
+      numPartitions: Int,
+      checksumAlgorithm: String): Array[RowBasedChecksum] = {
+    val rowBasedChecksums: Array[RowBasedChecksum] = new 
Array[RowBasedChecksum](numPartitions)
+    for (i <- 0 until numPartitions) {
+      rowBasedChecksums(i) = new 
OutputStreamRowBasedChecksum(checksumAlgorithm)
+    }
+    rowBasedChecksums
+  }
+
+  def getAggregatedChecksumValue(rowBasedChecksums: Array[RowBasedChecksum]): 
Long = {
+    val numPartitions: Int = if (rowBasedChecksums != null) 
rowBasedChecksums.length else 0
+    var aggregatedChecksum: Long = 0
+    if (numPartitions > 0) {
+      for (i <- 0 until numPartitions) {
+        aggregatedChecksum = aggregatedChecksum * 31 + 
rowBasedChecksums(i).getValue
+      }
+    }
+    return aggregatedChecksum

Review Comment:
   Done. Thanks.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -5724,6 +5724,21 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLED =
+    buildConf("spark.shuffle.orderIndependentChecksum.enabled")
+      .doc("Whether to calculate order independent checksum for the shuffle 
data or not. If " +
+        "enabled, Spark will calculate a checksum that is independent of the 
input row order for " +
+        "each mapper and returns the checksums from executors to driver. 
Different from the above" +
+        "checksum, the order independent remains the same even if the shuffle 
row order changes. " +
+        "While the above checksum is sensitive to shuffle data ordering to 
detect file " +
+        "corruption. This checksum is used to detect whether different task 
attempts of the same " +
+        "partition produce different output data or not (same set of keyValue 
pairs). In case " +
+        "the output data has changed across retries, Spark will need to retry 
all tasks of the " +
+        "consumer stages to avoid correctness issues.")
+      .version("4.1.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   Done



##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.shuffle.checksum
+
+import java.io.{ByteArrayOutputStream, ObjectOutputStream}
+import java.util.zip.Checksum
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper
+
+/**
+ * A class for computing checksum for input (key, value) pairs. The checksum 
is independent of
+ * the order of the input (key, value) pairs. It is done by computing a 
checksum for each row
+ * first, and then computing the XOR for all the row checksums.
+ */
+abstract class RowBasedChecksum() extends Serializable with Logging {
+  private var hasError: Boolean = false
+  private var checksumValue: Long = 0
+  /** Returns the checksum value computed. Tt returns the default checksum 
value (0) if there
+   * are any errors encountered during the checksum computation.
+   */
+  def getValue: Long = {
+    if (!hasError) checksumValue else 0
+  }
+
+  /** Updates the row-based checksum with the given (key, value) pair */
+  def update(key: Any, value: Any): Unit = {
+    if (!hasError) {
+      try {
+        val rowChecksumValue = calculateRowChecksum(key, value)
+        checksumValue = checksumValue ^ rowChecksumValue
+      } catch {
+        case NonFatal(e) =>
+          logInfo("Checksum computation encountered error: ", e)
+          hasError = true
+      }
+    }
+  }
+
+  /** Computes and returns the checksum value for the given (key, value) pair 
*/
+  protected def calculateRowChecksum(key: Any, value: Any): Long
+}
+
+/**
+ * A Concrete implementation of RowBasedChecksum. The checksum for each row is
+ * computed by first converting the (key, value) pair to byte array using 
OutputStreams,
+ * and then computing the checksum for the byte array.
+ *
+ * @param checksumAlgorithm the algorithm used for computing checksum.
+ */
+class OutputStreamRowBasedChecksum(checksumAlgorithm: String)
+  extends RowBasedChecksum() {
+
+  /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */
+  final private class MyByteArrayOutputStream(size: Int)
+    extends ByteArrayOutputStream(size) {
+    def getBuf: Array[Byte] = buf
+  }
+
+  private val DEFAULT_INITIAL_SER_BUFFER_SIZE = 32 * 1024
+
+  @transient private lazy val serBuffer =
+    new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE)
+  @transient private lazy val objOut = new ObjectOutputStream(serBuffer)
+
+  @transient
+  protected lazy val checksum: Checksum =
+    ShuffleChecksumHelper.getChecksumByAlgorithm(checksumAlgorithm)
+
+  override protected def calculateRowChecksum(key: Any, value: Any): Long = {
+    assert(checksum != null, "Checksum is null")
+
+    // Converts the (key, value) pair into byte array.
+    objOut.reset()
+    serBuffer.reset()
+    objOut.writeObject((key, value))
+    objOut.flush()
+    serBuffer.flush()
+
+    // Computes and returns the checksum for the byte array.
+    checksum.reset()
+    checksum.update(serBuffer.getBuf, 0, serBuffer.size())
+    checksum.getValue
+  }
+}
+
+object RowBasedChecksum {
+  def createPartitionRowBasedChecksums(
+      numPartitions: Int,
+      checksumAlgorithm: String): Array[RowBasedChecksum] = {
+    val rowBasedChecksums: Array[RowBasedChecksum] = new 
Array[RowBasedChecksum](numPartitions)
+    for (i <- 0 until numPartitions) {
+      rowBasedChecksums(i) = new 
OutputStreamRowBasedChecksum(checksumAlgorithm)
+    }
+    rowBasedChecksums

Review Comment:
   Done. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to