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

    https://github.com/apache/spark/pull/15702#discussion_r86067616
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala
 ---
    @@ -104,85 +110,105 @@ case class StateStoreSaveExec(
     
       override protected def doExecute(): RDD[InternalRow] = {
         metrics // force lazy init at driver
    -    assert(returnAllStates.nonEmpty,
    -      "Incorrect planning in IncrementalExecution, returnAllStates have 
not been set")
    -    val saveAndReturnFunc = if (returnAllStates.get) saveAndReturnAll _ 
else saveAndReturnUpdated _
    +    assert(outputMode.nonEmpty,
    +      "Incorrect planning in IncrementalExecution, outputMode has not been 
set")
    +
         child.execute().mapPartitionsWithStateStore(
           getStateId.checkpointLocation,
           operatorId = getStateId.operatorId,
           storeVersion = getStateId.batchId,
           keyExpressions.toStructType,
           child.output.toStructType,
           sqlContext.sessionState,
    -      Some(sqlContext.streams.stateStoreCoordinator)
    -    )(saveAndReturnFunc)
    +      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")
    +
    +        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
    +            }
    +            store.commit()
    +            numTotalStateRows += store.numKeys()
    +            store.iterator().map { case (k, v) =>
    +              numOutputRows += 1
    +              v.asInstanceOf[InternalRow]
    +            }
    +
    +          // Update and output only rows being evicted from the StateStore
    +          case Some(Append) =>
    +            while (iter.hasNext) {
    +              val row = iter.next().asInstanceOf[UnsafeRow]
    +              val key = getKey(row)
    +              store.put(key.copy(), row.copy())
    +              numUpdatedStateRows += 1
    +            }
    +
    +            val watermarkAttribute =
    +              
keyExpressions.find(_.metadata.contains(EventTimeWatermark.delayKey)).get
    +            // If we are evicting based on a window, use the end of the 
window.  Otherwise just
    +            // use the attribute itself.
    +            val evictionExpression =
    +              if (watermarkAttribute.dataType.isInstanceOf[StructType]) {
    +                LessThanOrEqual(
    +                  GetStructField(watermarkAttribute, 1),
    +                  Literal(eventTimeWatermark.get * 1000))
    +              } else {
    +                LessThanOrEqual(
    +                  watermarkAttribute,
    +                  Literal(eventTimeWatermark.get * 1000))
    +              }
    +
    +            logInfo(s"Filtering state store on: $evictionExpression")
    +            val predicate = newPredicate(evictionExpression, 
keyExpressions)
    +            store.remove(predicate)
    +
    +            store.commit()
    +
    +            numTotalStateRows += store.numKeys()
    +            store.updates().filter(_.isInstanceOf[ValueRemoved]).map { 
removed =>
    +              numOutputRows += 1
    +              removed.value.asInstanceOf[InternalRow]
    +            }
    +
    +          // Update and output modified rows from the StateStore.
    +          case Some(Update) =>
    --- End diff --
    
    I'm not clear on why the semantics of Update mean that watermarks shouldn't 
be used to remove state.


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