Github user tdas commented on a diff in the pull request: https://github.com/apache/spark/pull/9256#discussion_r44205421 --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/TrackStateRDD.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.rdd + +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import org.apache.spark.rdd.{MapPartitionsRDD, RDD} +import org.apache.spark.streaming.{StateImpl, State} +import org.apache.spark.streaming.util.{EmptyStateMap, StateMap} +import org.apache.spark.util.Utils +import org.apache.spark._ + +private[streaming] case class TrackStateRDDRecord[K, S, T]( + var stateMap: StateMap[K, S], var emittedRecords: Seq[T]) { + /* + private def writeObject(outputStream: ObjectOutputStream): Unit = { + outputStream.writeObject(stateMap) + outputStream.writeInt(emittedRecords.size) + val iterator = emittedRecords.iterator + while(iterator.hasNext) { + outputStream.writeObject(iterator.next) + } + } + + private def readObject(inputStream: ObjectInputStream): Unit = { + stateMap = inputStream.readObject().asInstanceOf[StateMap[K, S]] + val numEmittedRecords = inputStream.readInt() + val array = new Array[T](numEmittedRecords) + var i = 0 + while(i < numEmittedRecords) { + array(i) = inputStream.readObject().asInstanceOf[T] + } + emittedRecords = array.toSeq + }*/ +} + + +private[streaming] class TrackStateRDDPartition( + idx: Int, + @transient private var prevStateRDD: RDD[_], + @transient private var partitionedDataRDD: RDD[_]) extends Partition { + + private[rdd] var previousSessionRDDPartition: Partition = null + private[rdd] 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() + } +} + + +/** + * RDD storing the keyed-state of trackStateByKey and corresponding emitted records. + * Each partition of this RDD has a single record that contains a StateMap storing + */ +private[streaming] class TrackStateRDD[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag]( + 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]]( + partitionedDataRDD.sparkContext, + List( + new OneToOneDependency[TrackStateRDDRecord[K, S, T]](prevStateRDD), + new OneToOneDependency(partitionedDataRDD)) + ) { + + @volatile private var doFullScan = false + + require(prevStateRDD.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) + + val newStateMap = if (prevStateRDDIterator.hasNext) { + prevStateRDDIterator.next().stateMap.copy() + } else { + new EmptyStateMap[K, S]() + } + + 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 + } + + if (doFullScan) { + if (timeoutThresholdTime.isDefined) { --- End diff -- done.
--- 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