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


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -1650,6 +1650,21 @@ package object config {
         s"The buffer size must be greater than 0 and less than or equal to 
${Int.MaxValue}.")
       .createWithDefault(4096)
 
+  private[spark] val SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLED =
+    ConfigBuilder("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" +

Review Comment:
   nit: need space at end of string on this line



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -1650,6 +1650,21 @@ package object config {
         s"The buffer size must be greater than 0 and less than or equal to 
${Int.MaxValue}.")
       .createWithDefault(4096)
 
+  private[spark] val SHUFFLE_ORDER_INDEPENDENT_CHECKSUM_ENABLED =
+    ConfigBuilder("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 " +

Review Comment:
   what is "above checksum" referring to?



##########
core/src/main/java/org/apache/spark/shuffle/checksum/RowBasedChecksum.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.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
+import org.apache.spark.util.ExposedBufferByteArrayOutputStream
+
+/**
+ * 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) =>
+          logError("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.
+ * Note that this checksum computation is very expensive, and it is used only 
in tests
+ * in the core component. A much cheaper implementation of RowBasedChecksum is 
in
+ * UnsafeRowChecksum.
+ *
+ * @param checksumAlgorithm the algorithm used for computing checksum.
+ */
+class OutputStreamRowBasedChecksum(checksumAlgorithm: String)

Review Comment:
   just a question: I don't see this used anywhere except in tests, why not 
have it in `core/src/test` instead?



##########
core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala:
##########
@@ -58,6 +58,12 @@ private[spark] sealed trait MapStatus extends 
ShuffleOutputStatus {
    * partitionId of the task or taskContext.taskAttemptId is used.
    */
   def mapId: Long
+
+  /**
+   * The checksum value of this shuffle map task, which can be used to 
evaluate whether the
+   * output data have changed across different map task retries.

Review Comment:
   `output data have` -> `output data has`



-- 
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