Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18107#discussion_r119016141
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
 ---
    @@ -165,54 +189,88 @@ case class StateStoreSaveExec(
         child.execute().mapPartitionsWithStateStore(
           getStateId.checkpointLocation,
           getStateId.operatorId,
    +      storeName = "default",
           getStateId.batchId,
           keyExpressions.toStructType,
           child.output.toStructType,
    +      indexOrdinal = None,
           sqlContext.sessionState,
           Some(sqlContext.streams.stateStoreCoordinator)) { (store, iter) =>
             val getKey = GenerateUnsafeProjection.generate(keyExpressions, 
child.output)
             val numOutputRows = longMetric("numOutputRows")
             val numTotalStateRows = longMetric("numTotalStateRows")
             val numUpdatedStateRows = longMetric("numUpdatedStateRows")
    +        val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
    +        val allRemovalsTimeMs = longMetric("allRemovalsTimeMs")
    +        val commitTimeMs = longMetric("commitTimeMs")
     
             outputMode match {
               // Update and output all rows in the StateStore.
               case Some(Complete) =>
    -            while (iter.hasNext) {
    -              val row = iter.next().asInstanceOf[UnsafeRow]
    -              val key = getKey(row)
    -              store.put(key.copy(), row.copy())
    -              numUpdatedStateRows += 1
    +            allUpdatesTimeMs += timeTakenMs {
    +              while (iter.hasNext) {
    +                val row = iter.next().asInstanceOf[UnsafeRow]
    +                val key = getKey(row)
    +                store.put(key, row)
    +                numUpdatedStateRows += 1
    +              }
    +            }
    +            allRemovalsTimeMs += 0
    +            commitTimeMs += timeTakenMs {
    +              store.commit()
                 }
    -            store.commit()
                 numTotalStateRows += store.numKeys()
    -            store.iterator().map { case (k, v) =>
    +            store.iterator().map { case UnsafeRowPair(_, v) =>
                   numOutputRows += 1
                   v.asInstanceOf[InternalRow]
                 }
     
               // Update and output only rows being evicted from the StateStore
    +          // Assumption: watermark predicates must be non-empty if append 
mode is allowed
               case Some(Append) =>
    -            while (iter.hasNext) {
    -              val row = iter.next().asInstanceOf[UnsafeRow]
    -              val key = getKey(row)
    -              store.put(key.copy(), row.copy())
    -              numUpdatedStateRows += 1
    +            allUpdatesTimeMs += timeTakenMs {
    +              val filteredIter = iter.filter(row => 
!watermarkPredicateForData.get.eval(row))
    +              while (filteredIter.hasNext) {
    +                val row = filteredIter.next().asInstanceOf[UnsafeRow]
    +                val key = getKey(row)
    +                store.put(key, row)
    +                numUpdatedStateRows += 1
    +              }
                 }
     
    -            // Assumption: Append mode can be done only when watermark has 
been specified
    -            store.remove(watermarkPredicateForKeys.get.eval _)
    -            store.commit()
    +            val removalStartTime = System.currentTimeMillis
    +            val rangeIter = store.getRange(None, None)
    +
    +            new NextIterator[InternalRow] {
    +              override protected def getNext(): InternalRow = {
    +                var removedValueRow: InternalRow = null
    +                while(rangeIter.hasNext && removedValueRow == null) {
    +                  val UnsafeRowPair(keyRow, valueRow) = rangeIter.next()
    --- End diff --
    
    Case class's `unapply` will create a `Tuple`. You should not use this Scala 
syntactic sugar :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to