You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "zsxwing (via GitHub)" <gi...@apache.org> on 2023/03/03 09:09:14 UTC

[GitHub] [spark] zsxwing commented on a diff in pull request #39931: [SPARK-42376][SS] Introduce watermark propagation among operators

zsxwing commented on code in PR #39931:
URL: https://github.com/apache/spark/pull/39931#discussion_r1124167428


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -216,63 +222,140 @@ class IncrementalExecution(
           keys,
           child,
           Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None)
 
       case m: FlatMapGroupsWithStateExec =>
         // We set this to true only for the first batch of the streaming query.
         val hasInitialState = (currentBatchId == 0L && m.hasInitialState)
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None,
           hasInitialState = hasInitialState
         )
 
       case m: FlatMapGroupsInPandasWithStateExec =>
         m.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           batchTimestampMs = Some(offsetSeqMetadata.batchTimestampMs),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction)
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
         )
 
       case j: StreamingSymmetricHashJoinExec =>
         j.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
-          eventTimeWatermarkForLateEvents = Some(eventTimeWatermarkForLateEvents),
-          eventTimeWatermarkForEviction = Some(eventTimeWatermarkForEviction),
-          stateWatermarkPredicates =
-            StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
-              j.left.output, j.right.output, j.leftKeys, j.rightKeys, j.condition.full,
-              Some(eventTimeWatermarkForEviction)))
+          eventTimeWatermarkForLateEvents = None,
+          eventTimeWatermarkForEviction = None
+        )
 
       case l: StreamingGlobalLimitExec =>
         l.copy(
           stateInfo = Some(nextStatefulOperationStateInfo),
           outputMode = Some(outputMode))
+    }
+  }
 
-      case StreamingLocalLimitExec(limit, child) if hasNoStatefulOp(child) =>
-        // Optimize limit execution by replacing StreamingLocalLimitExec (consumes the iterator
-        // completely) to LocalLimitExec (does not consume the iterator) when the child plan has
-        // no stateful operator (i.e., consuming the iterator is not needed).
-        LocalLimitExec(limit, child)
+  val watermarkPropagationRule = new Rule[SparkPlan] {
+    private def simulateWatermarkPropagation(plan: SparkPlan): Unit = {
+      val watermarkForPrevBatch = prevOffsetSeqMetadata.map(_.batchWatermarkMs).getOrElse(0L)
+      val watermarkForCurrBatch = offsetSeqMetadata.batchWatermarkMs
+
+      // This is to simulate watermark propagation for late events.
+      watermarkPropagator.propagate(currentBatchId - 1, plan, watermarkForPrevBatch)
+      // This is to simulate watermark propagation for eviction.
+      watermarkPropagator.propagate(currentBatchId, plan, watermarkForCurrBatch)
+    }
+
+    private def inputWatermarkForLateEvents(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForLateEvents(currentBatchId,
+        stateInfo.operatorId))
+    }
+
+    private def inputWatermarkForEviction(stateInfo: StatefulOperatorStateInfo): Option[Long] = {
+      Some(watermarkPropagator.getInputWatermarkForEviction(currentBatchId, stateInfo.operatorId))
+    }
+
+    override def apply(plan: SparkPlan): SparkPlan = {
+      simulateWatermarkPropagation(plan)
+      plan transform {
+        case s: StateStoreSaveExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: SessionWindowStateStoreSaveExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: SessionWindowStateStoreRestoreExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case s: StreamingDeduplicateExec if s.stateInfo.isDefined =>
+          s.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(s.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(s.stateInfo.get)
+          )
+
+        case m: FlatMapGroupsWithStateExec if m.stateInfo.isDefined =>
+          m.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(m.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
+          )
+
+        case m: FlatMapGroupsInPandasWithStateExec if m.stateInfo.isDefined =>
+          m.copy(
+            eventTimeWatermarkForLateEvents = inputWatermarkForLateEvents(m.stateInfo.get),
+            eventTimeWatermarkForEviction = inputWatermarkForEviction(m.stateInfo.get)
+          )
+
+        case j: StreamingSymmetricHashJoinExec =>
+          val iwLateEvents = inputWatermarkForLateEvents(j.stateInfo.get)
+          val iwEviction = inputWatermarkForEviction(j.stateInfo.get)
+          j.copy(
+            eventTimeWatermarkForLateEvents = iwLateEvents,
+            eventTimeWatermarkForEviction = iwEviction,
+            stateWatermarkPredicates =
+              StreamingSymmetricHashJoinHelper.getStateWatermarkPredicates(
+                j.left.output, j.right.output, j.leftKeys, j.rightKeys, j.condition.full,
+                iwEviction, !allowMultipleStatefulOperators)
+          )
+      }
     }
   }
 
-  override def preparations: Seq[Rule[SparkPlan]] = state +: super.preparations
+  override def preparations: Seq[Rule[SparkPlan]] = Seq(
+    shufflePartitionsRule,

Review Comment:
   Is splitting rules just to improve readability? This would traverse the plan tree for 3 more passes. This is probably okay as people usually don't write complicated streaming queries.
      
   Batch queries are using a cheap API [transformUpWithPruning](https://github.com/apache/spark/blob/6874fd1b897680263ffbb3d06f716179aca8e4c566/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala#L557) to traverse tree. Would be great to keep this in mind and consider doing this in future if we need to add more rules.
   



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/**
+ * Interface for propagating watermark. The implementation is not required to be thread-safe,
+ * as all methods are expected to be called from the query execution thread.
+ * (The guarantee may change on further improvements on Structured Streaming - update
+ * implementations if we change the guarantee.)
+ */
+sealed trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+object NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  // We use treemap to sort the key (batchID) and evict old batch IDs efficiently.
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceOutputWatermark(input watermark)`.
+ *
+ *      @see [[StateStoreWriter.produceOutputWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  // We use treemap to sort the key (batchID) and evict old batch IDs efficiently.
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  // contains the association for batchId -> (stateful operator ID -> input watermark)
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Option[Long]]] =
+    mutable.Map[Long, Map[Long, Option[Long]]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  /**
+   * Retrieve the available input watermarks for specific node in the plan. Every child will
+   * produce an output watermark, which we capture as input watermark. If the child provides
+   * default watermark value (no watermark info), it is excluded.
+   */
+  private def getInputWatermarks(
+      node: SparkPlan,
+      nodeToOutputWatermark: mutable.Map[Int, Option[Long]]): Seq[Long] = {
+    node.children.flatMap { child =>
+      nodeToOutputWatermark.getOrElse(child.id, {
+        throw new IllegalStateException(
+          s"watermark for the node ${child.id} should be registered")
+      })
+      // Since we use flatMap here, this will exclude children from watermark calculation
+      // which don't have watermark information.
+    }
+  }
+
+  private def doSimulate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {

Review Comment:
   Can we skip this heavy work if the query doesn't define a watermark? Most streaming queries are in this case, and don't need to run this in each micro batch.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:
##########
@@ -212,6 +212,8 @@ class MicroBatchExecution(
     logInfo(s"Query $prettyIdString was stopped")
   }
 
+  private val watermarkPropagator = WatermarkPropagator(sparkSession.sessionState.conf)

Review Comment:
   How do we recover `WatermarkPropagator` for a query restart?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.streaming
+
+import java.{util => jutil}
+
+import scala.collection.mutable
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/**
+ * Interface for propagating watermark. The implementation is not required to be thread-safe,
+ * as all methods are expected to be called from the query execution thread.
+ * (The guarantee may change on further improvements on Structured Streaming - update
+ * implementations if we change the guarantee.)
+ */
+sealed trait WatermarkPropagator {
+  /**
+   * Request to propagate watermark among operators based on origin watermark value. The result
+   * should be input watermark per stateful operator, which Spark will request the value by calling
+   * getInputWatermarkXXX with operator ID.
+   *
+   * It is recommended for implementation to cache the result, as Spark can request the propagation
+   * multiple times with the same batch ID and origin watermark value.
+   */
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit
+
+  /** Provide the calculated input watermark for late events for given stateful operator. */
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long
+
+  /** Provide the calculated input watermark for eviction for given stateful operator. */
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long
+
+  /**
+   * Request to clean up cached result on propagation. Spark will call this method when the given
+   * batch ID will be likely to be not re-executed.
+   */
+  def purge(batchId: Long): Unit
+}
+
+/**
+ * Do nothing. This is dummy implementation to help creating a dummy IncrementalExecution instance.
+ */
+object NoOpWatermarkPropagator extends WatermarkPropagator {
+  def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {}
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long = Long.MinValue
+  def purge(batchId: Long): Unit = {}
+}
+
+/**
+ * This implementation uses a single global watermark for late events and eviction.
+ *
+ * This implementation provides the behavior before Structured Streaming supports multiple stateful
+ * operators. (prior to SPARK-40925) This is only used for compatibility mode.
+ */
+class UseSingleWatermarkPropagator extends WatermarkPropagator {
+  // We use treemap to sort the key (batchID) and evict old batch IDs efficiently.
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  override def propagate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    if (batchId < 0) {
+      // no-op
+    } else if (isInitialized(batchId)) {
+      val cached = batchIdToWatermark.get(batchId)
+      assert(cached == originWatermark,
+        s"Watermark has been changed for the same batch ID! Batch ID: $batchId, " +
+          s"Value in cache: $cached, value given: $originWatermark")
+    } else {
+      batchIdToWatermark.put(batchId, originWatermark)
+    }
+  }
+
+  private def getInputWatermark(batchId: Long, stateOpId: Long): Long = {
+    if (batchId < 0) {
+      0
+    } else {
+      assert(isInitialized(batchId), s"Watermark for batch ID $batchId is not yet set!")
+      batchIdToWatermark.get(batchId)
+    }
+  }
+
+  def getInputWatermarkForLateEvents(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  def getInputWatermarkForEviction(batchId: Long, stateOpId: Long): Long =
+    getInputWatermark(batchId, stateOpId)
+
+  override def purge(batchId: Long): Unit = {
+    val keyIter = batchIdToWatermark.keySet().iterator()
+    var stopIter = false
+    while (keyIter.hasNext && !stopIter) {
+      val currKey = keyIter.next()
+      if (currKey <= batchId) {
+        keyIter.remove()
+      } else {
+        stopIter = true
+      }
+    }
+  }
+}
+
+/**
+ * This implementation simulates propagation of watermark among operators.
+ *
+ * The simulation algorithm traverses the physical plan tree via post-order (children first) to
+ * calculate (input watermark, output watermark) for all nodes.
+ *
+ * For each node, below logic is applied:
+ *
+ * - Input watermark for specific node is decided by `min(input watermarks from all children)`.
+ *   -- Children providing no input watermark (DEFAULT_WATERMARK_MS) are excluded.
+ *   -- If there is no valid input watermark from children, input watermark = DEFAULT_WATERMARK_MS.
+ * - Output watermark for specific node is decided as following:
+ *   -- watermark nodes: origin watermark value
+ *      This could be individual origin watermark value, but we decide to retain global watermark
+ *      to keep the watermark model be simple.
+ *   -- stateless nodes: same as input watermark
+ *   -- stateful nodes: the return value of `op.produceOutputWatermark(input watermark)`.
+ *
+ *      @see [[StateStoreWriter.produceOutputWatermark]]
+ *
+ * Note that this implementation will throw an exception if watermark node sees a valid input
+ * watermark from children, meaning that we do not support re-definition of watermark.
+ *
+ * Once the algorithm traverses the physical plan tree, the association between stateful operator
+ * and input watermark will be constructed. Spark will request the input watermark for specific
+ * stateful operator, which this implementation will give the value from the association.
+ */
+class PropagateWatermarkSimulator extends WatermarkPropagator with Logging {
+  // We use treemap to sort the key (batchID) and evict old batch IDs efficiently.
+  private val batchIdToWatermark: jutil.TreeMap[Long, Long] = new jutil.TreeMap[Long, Long]()
+
+  // contains the association for batchId -> (stateful operator ID -> input watermark)
+  private val inputWatermarks: mutable.Map[Long, Map[Long, Option[Long]]] =
+    mutable.Map[Long, Map[Long, Option[Long]]]()
+
+  private def isInitialized(batchId: Long): Boolean = batchIdToWatermark.containsKey(batchId)
+
+  /**
+   * Retrieve the available input watermarks for specific node in the plan. Every child will
+   * produce an output watermark, which we capture as input watermark. If the child provides
+   * default watermark value (no watermark info), it is excluded.
+   */
+  private def getInputWatermarks(
+      node: SparkPlan,
+      nodeToOutputWatermark: mutable.Map[Int, Option[Long]]): Seq[Long] = {
+    node.children.flatMap { child =>
+      nodeToOutputWatermark.getOrElse(child.id, {
+        throw new IllegalStateException(
+          s"watermark for the node ${child.id} should be registered")
+      })
+      // Since we use flatMap here, this will exclude children from watermark calculation
+      // which don't have watermark information.
+    }
+  }
+
+  private def doSimulate(batchId: Long, plan: SparkPlan, originWatermark: Long): Unit = {
+    val statefulOperatorIdToNodeId = mutable.HashMap[Long, Int]()
+    val nodeToOutputWatermark = mutable.HashMap[Int, Option[Long]]()

Review Comment:
   If we are doing a batch-streaming join, do we still calculate watermarks for the batch side even if they are always empty?



-- 
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-unsubscribe@spark.apache.org

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


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