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

    https://github.com/apache/spark/pull/9256#discussion_r43206878
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/dstream/TrackeStateDStream.scala
 ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.streaming.dstream
    +
    +import java.io.{IOException, ObjectOutputStream}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import scala.reflect.ClassTag
    +
    +import org.apache.spark._
    +import org.apache.spark.rdd.{EmptyRDD, RDD}
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming._
    +import org.apache.spark.streaming.util.StateMap
    +import org.apache.spark.util.Utils
    +
    +private[streaming] case class TrackStateRDDRecord[K: ClassTag, S: 
ClassTag, T: ClassTag](
    +    stateMap: StateMap[K, S], emittedRecords: Seq[T])
    +
    +
    +private[streaming] class TrackStateRDDPartition(
    +    idx: Int,
    +    @transient private var prevStateRDD: RDD[_],
    +    @transient private var partitionedDataRDD: RDD[_]) extends Partition {
    +
    +  private[dstream] var previousSessionRDDPartition: Partition = null
    +  private[dstream] var partitionedDataRDDPartition: Partition = null
    +
    +  override def index: Int = idx
    +  override def hashCode(): Int = idx
    +
    +  @throws(classOf[IOException])
    +  private def writeObject(oos: ObjectOutputStream): Unit = 
Utils.tryOrIOException {
    +    // Update the reference to parent split at the time of task 
serialization
    +    previousSessionRDDPartition = prevStateRDD.partitions(index)
    +    partitionedDataRDDPartition = partitionedDataRDD.partitions(index)
    +    oos.defaultWriteObject()
    +  }
    +}
    +
    +private[streaming] class TrackStateRDD[K: ClassTag, V: ClassTag, S: 
ClassTag, T: ClassTag](
    +    _sc: SparkContext,
    +    private var prevStateRDD: RDD[TrackStateRDDRecord[K, S, T]],
    +    private var partitionedDataRDD: RDD[(K, V)],
    +    trackingFunction: (K, Option[V], State[S]) => Option[T],
    +    currentTime: Long, timeoutThresholdTime: Option[Long]
    +  ) extends RDD[TrackStateRDDRecord[K, S, T]](
    +    _sc,
    +    List(
    +      new OneToOneDependency[TrackStateRDDRecord[K, S, T]](prevStateRDD),
    +      new OneToOneDependency(partitionedDataRDD))
    +  ) {
    +
    +  @volatile private var doFullScan = false
    +
    +  require(partitionedDataRDD.partitioner.nonEmpty)
    +  require(partitionedDataRDD.partitioner == prevStateRDD.partitioner)
    +
    +  override val partitioner = prevStateRDD.partitioner
    +
    +  override def checkpoint(): Unit = {
    +    super.checkpoint()
    +    doFullScan = true
    +  }
    +
    +  override def compute(
    +      partition: Partition, context: TaskContext): 
Iterator[TrackStateRDDRecord[K, S, T]] = {
    +
    +    val stateRDDPartition = partition.asInstanceOf[TrackStateRDDPartition]
    +    val prevStateRDDIterator = prevStateRDD.iterator(
    +      stateRDDPartition.previousSessionRDDPartition, context)
    +    val dataIterator = partitionedDataRDD.iterator(
    +      stateRDDPartition.partitionedDataRDDPartition, context)
    +    if (!prevStateRDDIterator.hasNext) {
    +      throw new SparkException(s"Could not find state map in previous 
state RDD")
    +    }
    +
    +    val newStateMap = prevStateRDDIterator.next().stateMap.copy()
    +    val emittedRecords = new ArrayBuffer[T]
    +
    +    val wrappedState = new StateImpl[S]()
    +
    +    dataIterator.foreach { case (key, value) =>
    +      wrappedState.wrap(newStateMap.get(key))
    +      val emittedRecord = trackingFunction(key, Some(value), wrappedState)
    +      if (wrappedState.isRemoved) {
    +        newStateMap.remove(key)
    +      } else if (wrappedState.isUpdated) {
    +        newStateMap.put(key, wrappedState.get(), currentTime)
    +      }
    +      emittedRecords ++= emittedRecord
    --- End diff --
    
    I think you have to materialize the emitted records. Since the combo of 
[StateMap, emitted records] is a single record for the StateRDD, when the 
"record" is computed AND persisted, all the necessary information needs to be 
available in memory, and not require recomputing. Once persisted, the record 
should be usable any number of times. So if the emitted data *inside* that 
record is an iterator, it can not be reused any number of times. Which breaks 
the RDD's specifications.


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to