You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/02 06:00:07 UTC

[GitHub] [spark] viirya commented on a change in pull request #26935: [SPARK-30294][SS] Explicitly defines read-only StateStore and optimize for HDFSBackedStateStore

viirya commented on a change in pull request #26935:
URL: https://github.com/apache/spark/pull/26935#discussion_r498629306



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala
##########
@@ -29,6 +29,57 @@ import org.apache.spark.sql.internal.SessionState
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.SerializableConfiguration
 
+/**
+ * An RDD that allows computations to be executed against [[ReadOnlyStateStore]]s. It
+ * uses the [[StateStoreCoordinator]] to get the locations of loaded state stores
+ * and use that as the preferred locations.
+ */
+class ReadOnlyStateStoreRDD[T: ClassTag, U: ClassTag](
+    dataRDD: RDD[T],
+    storeReadFunction: (ReadOnlyStateStore, Iterator[T]) => Iterator[U],
+    checkpointLocation: String,
+    queryRunId: UUID,
+    operatorId: Long,
+    storeVersion: Long,
+    keySchema: StructType,
+    valueSchema: StructType,
+    indexOrdinal: Option[Int],
+    sessionState: SessionState,
+    @transient private val storeCoordinator: Option[StateStoreCoordinatorRef],
+    extraOptions: Map[String, String] = Map.empty) extends RDD[U](dataRDD) {
+
+  private val storeConf = new StateStoreConf(sessionState.conf, extraOptions)
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = dataRDD.context.broadcast(
+    new SerializableConfiguration(sessionState.newHadoopConf()))
+
+  override protected def getPartitions: Array[Partition] = dataRDD.partitions
+
+  /**
+   * Set the preferred location of each partition using the executor that has the related
+   * [[StateStoreProvider]] already loaded.
+   */
+  override def getPreferredLocations(partition: Partition): Seq[String] = {
+    val stateStoreProviderId = StateStoreProviderId(
+      StateStoreId(checkpointLocation, operatorId, partition.index),
+      queryRunId)
+    storeCoordinator.flatMap(_.getLocation(stateStoreProviderId)).toSeq
+  }
+
+  override def compute(partition: Partition, ctxt: TaskContext): Iterator[U] = {
+    val storeProviderId = StateStoreProviderId(
+      StateStoreId(checkpointLocation, operatorId, partition.index),
+      queryRunId)
+
+    val store = StateStore.getReadOnly(
+      storeProviderId, keySchema, valueSchema, indexOrdinal, storeVersion,

Review comment:
       Sounds too complicated, and it's also a bit weird and read-only store calls both abort and close. Let's keep with original abort and commit for now.




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

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