You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HeartSaVioR (via GitHub)" <gi...@apache.org> on 2023/10/18 06:07:01 UTC

[PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

HeartSaVioR opened a new pull request, #43425:
URL: https://github.com/apache/spark/pull/43425

   ### What changes were proposed in this pull request?
   
   This PR proposes to introduce a baseline implementation of state processor - reader.
   
   State processor is a new data source which enables reading and writing the state in the existing checkpoint with the batch query. Since we implement the feature as data source, we are leveraging the UX for DataFrame API which most users are already familiar with.
   
   Functionalities of the baseline implementation are following:
   
   * Specify a state store instance via store name (default: DEFAULT)
   * Specify a stateful operator via operator ID (default: 0)
   * Specify a batch ID (default: last committed)
   * Specify the source option joinSide to construct input rows in the state store for stream-stream join
     * It is still possible that users can read a specific state store instance from 4 instances in stream-stream join, which would be used mostly for debugging Spark itself
     * When this is enabled, the data source hides the internal column from the output.
   * Specify a metadata column (_partition_id)so that users can indicate the partition ID for the state row.
   
   ### Why are the changes needed?
   
   Please refer to the SPIP doc for rationale: https://docs.google.com/document/d/1_iVf_CIu2RZd3yWWF6KoRNlBiz5NbSIK0yThqG0EvPY/edit?usp=sharing
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, we are adding a new data source.
   
   ### How was this patch tested?
   
   New test suite.
   
   ### Was this patch authored or co-authored using generative AI tooling?
   
   No.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364339495


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2TestBase.scala:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.streaming._
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+trait StateDataSourceV2TestBase extends StreamTest with StateStoreMetricsTest {
+  import testImplicits._
+
+  protected def runCompositeKeyStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getCompositeKeyStreamingAggregationQuery(inputData)
+
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 to 5: _*),
+      CheckLastBatch(
+        (0, "Apple", 1, 0, 0, 0),
+        (1, "Banana", 1, 1, 1, 1),
+        (0, "Strawberry", 1, 2, 2, 2),
+        (1, "Apple", 1, 3, 3, 3),
+        (0, "Banana", 1, 4, 4, 4),
+        (1, "Strawberry", 1, 5, 5, 5)
+      ),
+      // batch 1
+      AddData(inputData, 6 to 10: _*),
+      // state also contains (1, "Strawberry", 1, 5, 5, 5) but not updated here
+      CheckLastBatch(
+        (0, "Apple", 2, 6, 6, 0), // 0, 6
+        (1, "Banana", 2, 8, 7, 1), // 1, 7
+        (0, "Strawberry", 2, 10, 8, 2), // 2, 8
+        (1, "Apple", 2, 12, 9, 3), // 3, 9
+        (0, "Banana", 2, 14, 10, 4) // 4, 10
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 3, 2, 1),
+      CheckLastBatch(
+        (1, "Banana", 3, 9, 7, 1), // 1, 7, 1
+        (0, "Strawberry", 3, 12, 8, 2), // 2, 8, 2
+        (1, "Apple", 3, 15, 9, 3) // 3, 9, 3
+      )
+    )
+  }
+
+  protected def getCompositeKeyStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, String, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 2 AS groupKey",
+        "(CASE value % 3 WHEN 0 THEN 'Apple' WHEN 1 THEN 'Banana' ELSE 'Strawberry' END) AS fruit")
+      .groupBy($"groupKey", $"fruit")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, String, Long, Long, Int, Int)]
+  }
+
+  protected def runLargeDataStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getLargeDataStreamingAggregationQuery(inputData)
+
+    // check with more data - leverage full partitions
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 until 20: _*),
+      CheckLastBatch(
+        (0, 2, 10, 10, 0), // 0, 10
+        (1, 2, 12, 11, 1), // 1, 11
+        (2, 2, 14, 12, 2), // 2, 12
+        (3, 2, 16, 13, 3), // 3, 13
+        (4, 2, 18, 14, 4), // 4, 14
+        (5, 2, 20, 15, 5), // 5, 15
+        (6, 2, 22, 16, 6), // 6, 16
+        (7, 2, 24, 17, 7), // 7, 17
+        (8, 2, 26, 18, 8), // 8, 18
+        (9, 2, 28, 19, 9) // 9, 19
+      ),
+      // batch 1
+      AddData(inputData, 20 until 40: _*),
+      CheckLastBatch(
+        (0, 4, 60, 30, 0), // 0, 10, 20, 30
+        (1, 4, 64, 31, 1), // 1, 11, 21, 31
+        (2, 4, 68, 32, 2), // 2, 12, 22, 32
+        (3, 4, 72, 33, 3), // 3, 13, 23, 33
+        (4, 4, 76, 34, 4), // 4, 14, 24, 34
+        (5, 4, 80, 35, 5), // 5, 15, 25, 35
+        (6, 4, 84, 36, 6), // 6, 16, 26, 36
+        (7, 4, 88, 37, 7), // 7, 17, 27, 37
+        (8, 4, 92, 38, 8), // 8, 18, 28, 38
+        (9, 4, 96, 39, 9) // 9, 19, 29, 39
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 0, 1, 2),
+      CheckLastBatch(
+        (0, 5, 60, 30, 0), // 0, 10, 20, 30, 0
+        (1, 5, 65, 31, 1), // 1, 11, 21, 31, 1
+        (2, 5, 70, 32, 2) // 2, 12, 22, 32, 2
+      )
+    )
+  }
+
+  protected def getLargeDataStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 10 AS groupKey")
+      .groupBy($"groupKey")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, Long, Long, Int, Int)]
+  }
+
+  protected def runDropDuplicatesQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val deduplicated = getDropDuplicatesQuery(inputData)
+
+    testStream(deduplicated, OutputMode.Append())(
+      StartStream(checkpointLocation = checkpointRoot),
+
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      AddData(inputData, 25), // Advance watermark to 15 secs, no-data-batch drops rows <= 15
+      CheckNewAnswer(25),
+      assertNumStateRows(total = 1, updated = 1),
+
+      AddData(inputData, 10), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25
+      CheckNewAnswer(45),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  protected def getDropDuplicatesQuery(inputData: MemoryStream[Int]): Dataset[Long] = {
+    inputData.toDS()
+      .withColumn("eventTime", timestamp_seconds($"value"))
+      .withWatermark("eventTime", "10 seconds")
+      .dropDuplicates()

Review Comment:
   Can we also add the test where columns are explicitly specified ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2TestBase.scala:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.streaming._
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+trait StateDataSourceV2TestBase extends StreamTest with StateStoreMetricsTest {
+  import testImplicits._
+
+  protected def runCompositeKeyStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getCompositeKeyStreamingAggregationQuery(inputData)
+
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 to 5: _*),
+      CheckLastBatch(
+        (0, "Apple", 1, 0, 0, 0),
+        (1, "Banana", 1, 1, 1, 1),
+        (0, "Strawberry", 1, 2, 2, 2),
+        (1, "Apple", 1, 3, 3, 3),
+        (0, "Banana", 1, 4, 4, 4),
+        (1, "Strawberry", 1, 5, 5, 5)
+      ),
+      // batch 1
+      AddData(inputData, 6 to 10: _*),
+      // state also contains (1, "Strawberry", 1, 5, 5, 5) but not updated here
+      CheckLastBatch(
+        (0, "Apple", 2, 6, 6, 0), // 0, 6
+        (1, "Banana", 2, 8, 7, 1), // 1, 7
+        (0, "Strawberry", 2, 10, 8, 2), // 2, 8
+        (1, "Apple", 2, 12, 9, 3), // 3, 9
+        (0, "Banana", 2, 14, 10, 4) // 4, 10
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 3, 2, 1),
+      CheckLastBatch(
+        (1, "Banana", 3, 9, 7, 1), // 1, 7, 1
+        (0, "Strawberry", 3, 12, 8, 2), // 2, 8, 2
+        (1, "Apple", 3, 15, 9, 3) // 3, 9, 3
+      )
+    )
+  }
+
+  protected def getCompositeKeyStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, String, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 2 AS groupKey",
+        "(CASE value % 3 WHEN 0 THEN 'Apple' WHEN 1 THEN 'Banana' ELSE 'Strawberry' END) AS fruit")
+      .groupBy($"groupKey", $"fruit")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, String, Long, Long, Int, Int)]
+  }
+
+  protected def runLargeDataStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getLargeDataStreamingAggregationQuery(inputData)
+
+    // check with more data - leverage full partitions
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 until 20: _*),
+      CheckLastBatch(
+        (0, 2, 10, 10, 0), // 0, 10
+        (1, 2, 12, 11, 1), // 1, 11
+        (2, 2, 14, 12, 2), // 2, 12
+        (3, 2, 16, 13, 3), // 3, 13
+        (4, 2, 18, 14, 4), // 4, 14
+        (5, 2, 20, 15, 5), // 5, 15
+        (6, 2, 22, 16, 6), // 6, 16
+        (7, 2, 24, 17, 7), // 7, 17
+        (8, 2, 26, 18, 8), // 8, 18
+        (9, 2, 28, 19, 9) // 9, 19
+      ),
+      // batch 1
+      AddData(inputData, 20 until 40: _*),
+      CheckLastBatch(
+        (0, 4, 60, 30, 0), // 0, 10, 20, 30
+        (1, 4, 64, 31, 1), // 1, 11, 21, 31
+        (2, 4, 68, 32, 2), // 2, 12, 22, 32
+        (3, 4, 72, 33, 3), // 3, 13, 23, 33
+        (4, 4, 76, 34, 4), // 4, 14, 24, 34
+        (5, 4, 80, 35, 5), // 5, 15, 25, 35
+        (6, 4, 84, 36, 6), // 6, 16, 26, 36
+        (7, 4, 88, 37, 7), // 7, 17, 27, 37
+        (8, 4, 92, 38, 8), // 8, 18, 28, 38
+        (9, 4, 96, 39, 9) // 9, 19, 29, 39
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 0, 1, 2),
+      CheckLastBatch(
+        (0, 5, 60, 30, 0), // 0, 10, 20, 30, 0
+        (1, 5, 65, 31, 1), // 1, 11, 21, 31, 1
+        (2, 5, 70, 32, 2) // 2, 12, 22, 32, 2
+      )
+    )
+  }
+
+  protected def getLargeDataStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 10 AS groupKey")
+      .groupBy($"groupKey")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, Long, Long, Int, Int)]
+  }
+
+  protected def runDropDuplicatesQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val deduplicated = getDropDuplicatesQuery(inputData)
+
+    testStream(deduplicated, OutputMode.Append())(
+      StartStream(checkpointLocation = checkpointRoot),
+
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      AddData(inputData, 25), // Advance watermark to 15 secs, no-data-batch drops rows <= 15
+      CheckNewAnswer(25),
+      assertNumStateRows(total = 1, updated = 1),
+
+      AddData(inputData, 10), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25
+      CheckNewAnswer(45),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  protected def getDropDuplicatesQuery(inputData: MemoryStream[Int]): Dataset[Long] = {

Review Comment:
   Can this be `private` ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388804304


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+/** An implementation of [[ScanBuilder]] for State Store data source. */
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {

Review Comment:
   Based to the state store API spec, we cannot get meaningful benefit from SupportsPushDownRequiredColumns and SupportsPushDownFilters, except the one, filter by partition ID which we only need to spin a state store for the specific partition. We can file a JIRA ticket as a follow-up.
   
   SupportsPushDownLimit could be something we can consider, though current state store implementations imply that iterator should have been fully consumed to close properly. Let's file a JIRA ticket as a follow-up anyway.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1790706683

   https://github.com/HeartSaVioR/spark/runs/18290152278
   All test failures seem to be related to instability of Spark connect tests.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1382392626


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,779 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.SparkUnsupportedOperationException
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: trying to read state data as stream") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      intercept[SparkUnsupportedOperationException] {
+        spark.readStream.format("statestore").load(tempDir.getAbsolutePath)
+          .writeStream.format("noop").start()
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "false")
+  }
+}
+
+class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "true")
+  }
+}
+
+abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Assertions {

Review Comment:
   I feel like 1 line class comment would end up describing the same we get from the suite name. Do you feel like the suite name does not self-describe, and if then which point you feel that way?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1770211883

   > it seems like we only expose partition Id as a metadata/internal column and that too only if its queried ? We don't seem to expose other cols such as batchId/operatorId etc. What is the reason for doing this ?
   
   The main usage for metadata column is to get data/row dependent internal information. batch ID and operator ID will be all same for all rows in the resulting DataFrame. Do we really need them?
   
   batchId might be still reasonable, as the default value is less obvious and users may want to know the exact value. It may be worth to discuss whether we want to add batchId or not. The default value of operator ID is simply obvious and users querying this should know the value in prior. Once they know that, withColumn is their friend.
   
   > for some of the queries such as join/FMGWS, it seems that we have different formats for v1/v2 and the user needs to query it differently within the selectExpr. How does the user discover these fields ? Is it possible to keep the source schema homogenous here ?
   
   Users can discover the schema before you execute the query. DataFrame.printSchema(). Specifically for stream-stream join, we expect users to specify the option joinSide and do not care about format version.
   
   I understand we are directly exposing the state as it is, which may be less familiar / friendly to users and concerning in UX perspective. My worrying for homogenizing is that we are going to require doing the same for every operator & every state format. Current implementation intentionally avoids to couple with the implementation detail of the operator. stream-stream join is just a one of the unfortunate case.
   
   But we could file an issue and discuss whether it is the right direction or not.
   
   > for join queries, what schema do we expose when a store name is explicitly specified vs not. I guess the ability to query a specific store name (esp the ones like right-keyToNumValues) is only really for debugging purposes in this case ? 
   
   For join queries with joinSide option being specified, all input columns should be exposed as value and equality joining columns should be exposed as key. If they omit the joinSide option, they need to specify the store name explicitly and then we expose the internal data structure as it is (although we do not expect users want to do that).
   
   > Also, for join queries, where do we add the internal metadata cols like partitionId - not sure I found that
   
   This is missing one. I'll leave a self-comment. Nice finding!
   
   > for the tests, not sure I saw a simulation for the expected use-cases. for eg - some tests where we keep the streaming query running for a few batches and assert for certain conditions/state values along the way. 
   
   So you meant reading the state from checkpoint without stopping streaming query, right?
   
   The guaranteeness for this use case is weak - the state reader never blocks the streaming query to clean up the version which state reader may decide to read. It's feasible we add test where the streaming query never goes beyond the version which triggers cleanup for state reader to read, but I'm wondering what we want to guarantee here. I'd probably explicitly mention that it is not suitable for the case where the streaming query is running and processing batches.
   
   All tests actually do that, right? We run these queries for a few batches, and verify the state values. I'm not sure what you are referring to.
   
   > Also, maybe around corruption detection where we artificially corrupt some values and show how the state reader can detect those ?
   
   I was looking into the option which provides the key/value as binary, but wanted to punt out to the further improvement. 
   
   Furthermore, I thought about this case more, and realized there are multiple cases; corruption happened but RocksDB encoder loaded into UnsafeRow anyway vs corruption happened which would break RocksDB encoder as well. For the first one we could probably still handle this on top of general state store API. For the second one, we will need to come up with modification of state store API, which brings the broken data separately with normal data. It seems to incur more work than we anticipated, hence not suitable for initial implementation.
   
   Probably the better approach is to have some similar option like corrupt record column in CSV/JSON data source. We could file an issue and look into it later.
   
   > For tests, should we also add some cases with additional startStream/stopStream clauses and verify that state read is working as expected even when batch recovery/restart cases are involved ?
   
   Not sure I understand this correctly. Mind elaborating which scenario you are looking for?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388798323


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"

Review Comment:
   This is actually for better UX, you pass the checkpoint path in load() and it works smoothly. We can address ambiguity in documentation.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID}' cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID}' cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))
+        .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+      if (storeName.isEmpty) {

Review Comment:
   Good point.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376952982


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {

Review Comment:
   Lets also add one for `storeName` is empty/invalid ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1780463851

   @anishshri-db Would you mind taking another look? Thanks in advance!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1378121683


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   I wonder how we handle the case where state metadata doesn't exist(if the query has never run in newer spark version), can we do it this way:
   Check whether state metadata file exists,
   if metadata exists, read the numColsPrefixKey from the metadata
   if metadata doesn't exist, we use 0 as the numColsPrefixKey and print a warning that this doesn't work for session window operator. Then let exception throw if it is session window operator.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "brkyvz (via GitHub)" <gi...@apache.org>.
brkyvz commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388765048


##########
sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala:
##########
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf
  * @since 2.0.0
  */
 @Stable
-class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) {
+class RuntimeConfig private[sql](val sqlConf: SQLConf = new SQLConf) {

Review Comment:
   hmm, @cloud-fan is this change safe? 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "brkyvz (via GitHub)" <gi...@apache.org>.
brkyvz commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388773469


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+/**
+ * An implementation of [[PartitionReaderFactory]] for State data source. This is used to support
+ * general read from a state store instance, rather than specific to the operator.
+ */
+class StatePartitionReaderFactory(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    schema: StructType) extends PartitionReaderFactory {
+
+  override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
+    new StatePartitionReader(storeConf, hadoopConf,
+      partition.asInstanceOf[StateStoreInputPartition], schema)
+  }
+}
+
+/**
+ * An implementation of [[PartitionReader]] for State data source. This is used to support
+ * general read from a state store instance, rather than specific to the operator.
+ */
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString,
+      partition.sourceOptions.operatorId, partition.partition, partition.sourceOptions.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an
+    //  information whether the state store uses prefix scan or not. We will have to add such
+    //  information to determine the right encoder/decoder for the data.
+    StateStore.getReadOnly(stateStoreProviderId, keySchema, valueSchema,
+      numColsPrefixKey = 0, version = partition.sourceOptions.batchId + 1, storeConf = storeConf,
+      hadoopConf = hadoopConf.value)
+  }
+
+  private lazy val iter = {

Review Comment:
   can you specify type please?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"

Review Comment:
   uber nit: can it be `state-store`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID}' cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID}' cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))
+        .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+      if (storeName.isEmpty) {

Review Comment:
   do you need to `trim` before calling `isEmpty`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+/** An implementation of [[ScanBuilder]] for State Store data source. */
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {

Review Comment:
   you don't want to support `SupportsPushDownRequiredColumns`, `SupportsPushDownFilters`, `SupportsPushDownLimit`, etc? 
   
   I guess these can be done in a follow up



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {

Review Comment:
   Can you extend the `DataSourceOptions` trait and register these as `newOptions`?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,779 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.SparkUnsupportedOperationException
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: trying to read state data as stream") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      intercept[SparkUnsupportedOperationException] {
+        spark.readStream.format("statestore").load(tempDir.getAbsolutePath)
+          .writeStream.format("noop").start()
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)

Review Comment:
   why not override `sqlConf`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala:
##########
@@ -88,8 +88,7 @@ class StateSchemaCompatibilityChecker(
   private def schemasCompatible(storedSchema: StructType, schema: StructType): Boolean =
     DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema)
 
-  // Visible for testing
-  private[sql] def readSchemaFile(): (StructType, StructType) = {

Review Comment:
   why do you need to remove the `private[sql]`? Isn't the whole code already in org.apache.spark.sql?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"

Review Comment:
   should we call this explicitly `checkpointLocation` so that users don't use the wrong path, e.g. stream source or target path?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala:
##########
@@ -185,6 +187,57 @@ class SymmetricHashJoinStateManager(
     }
   }
 
+  /**
+   * Perform a full scan to provide all available data.
+   *
+   * This produces an iterator over the (key, value, match) tuples. Callers are expected
+   * to consume fully to clean up underlying iterators correctly.

Review Comment:
   why is this the case? Don't you want to support limit pushdown?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,779 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.SparkUnsupportedOperationException
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {

Review Comment:
   I'd love to see all of these using the error framework



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")

Review Comment:
   can we use the error framework for all of these please?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)

Review Comment:
   Want to leverage the error framework: https://github.com/apache/spark/blob/master/common/utils/src/main/resources/error/error-classes.json



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID}' cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID}' cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))
+        .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+      if (storeName.isEmpty) {
+        throw new IllegalArgumentException(s"'${PARAM_STORE_NAME}' cannot be an empty string.")
+      }
+
+      val joinSide = try {
+        Option(options.get(PARAM_JOIN_SIDE))
+          .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+      } catch {
+        case _: NoSuchElementException =>
+          // convert to IllegalArgumentException
+          throw new IllegalArgumentException(s"Incorrect value of the option " +
+            s"'$PARAM_JOIN_SIDE'. Valid values are ${JoinSideValues.values.mkString(",")}")
+      }
+
+      if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+        throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+          s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+      }
+
+      StateSourceOptions(resolvedCpLocation, batchId, operatorId, storeName, joinSide)
+    }
+
+    private def resolvedCheckpointLocation(
+        hadoopConf: Configuration,
+        checkpointLocation: String): String = {
+      val checkpointPath = new Path(checkpointLocation)
+      val fs = checkpointPath.getFileSystem(hadoopConf)
+      checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+    }
+
+    private def getLastCommittedBatch(session: SparkSession, checkpointLocation: String): Long = {
+      val commitLog = new CommitLog(session, new Path(checkpointLocation, "commits").toString)

Review Comment:
   can we put `"commits"` and `"state"` as constants into a class like `object StreamingCheckpointConstants` or something?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1787660261

   @HeartSaVioR - test failure seems related ?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388820123


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala:
##########
@@ -88,8 +88,7 @@ class StateSchemaCompatibilityChecker(
   private def schemasCompatible(storedSchema: StructType, schema: StructType): Boolean =
     DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema)
 
-  // Visible for testing
-  private[sql] def readSchemaFile(): (StructType, StructType) = {

Review Comment:
   It's not about scoping. It is more about the representation that we are exposing these methods to others. `check()` method can be `private[sql]` as well but we don't, because the method is the entry point. We are adding more entry points, hence making them be public as well.
   (Although someone might argue that it's better to separate the methods for handling reading and writing schema file out to the separate class.)
   
   Technically saying, we should be able to make the entire class be package private, but we don't, because the package is considered as non-public one.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1805859712

   CI failed only in K8S integration test.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1367381248


##########
sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister:
##########
@@ -27,4 +27,5 @@ org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
 org.apache.spark.sql.execution.streaming.sources.RateStreamProvider
 org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider
 org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat
-org.apache.spark.sql.execution.streaming.sources.RatePerMicroBatchProvider
\ No newline at end of file
+org.apache.spark.sql.execution.streaming.sources.RatePerMicroBatchProvider
+org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2

Review Comment:
   Do we just want to rename to `StateDataSource` ? we already have v2 in the package path ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1773793847

   https://github.com/HeartSaVioR/spark/runs/17924716889
   
   ```
   ProtobufCatalystDataConversionSuite.single StructType(StructField(bytes_type,BinaryType,true)) with seed 115
   java.lang.NullPointerException: null
   ```
   
   @rangadi 
   The test doesn't seem to be stable. Shall we run the test suite with all combination for available seeds and available testing types and make sure all of them are passing?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376943396


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/** An implementation of [[Table]] with [[SupportsRead]] for State Store data source. */
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new IllegalStateException(s"Invalid schema is provided. Provided schema: $schema for " +
+      s"checkpoint location: ${sourceOptions.stateCheckpointLocation} , operatorId: " +
+      s"${sourceOptions.operatorId} , storeName: ${sourceOptions.storeName}, " +
+      s"joinSide: ${sourceOptions.joinSide}")
+  }
+
+  override def name(): String = {
+    val desc = s"StateTable " +
+      s"[stateCkptLocation=${sourceOptions.stateCheckpointLocation}]" +
+      s"[batchId=${sourceOptions.batchId}][operatorId=${sourceOptions.operatorId}]" +
+      s"[storeName=${sourceOptions.storeName}]"
+
+    if (sourceOptions.joinSide != JoinSideValues.none) {
+      desc + s"[joinSide=${sourceOptions.joinSide}]"
+    } else {
+      desc
+    }
+  }
+
+  override def capabilities(): util.Set[TableCapability] = CAPABILITY
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+    new StateScanBuilder(session, schema, sourceOptions, stateConf)
+
+  override def properties(): util.Map[String, String] = Map.empty[String, String].asJava
+
+  private def isValidSchema(schema: StructType): Boolean = {
+    if (schema.fieldNames.toSeq != Seq("key", "value")) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "key").isInstanceOf[StructType]) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "value").isInstanceOf[StructType]) {
+      false
+    } else {
+      true
+    }
+  }
+
+  override def metadataColumns(): Array[MetadataColumn] = METADATA_COLUMNS.toArray
+}
+
+object StateTable {

Review Comment:
   Maybe add a 1-2 line comment describing how/where this is used ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1378121683


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   I wonder how we handle the case where state metadata doesn't exist(if the query has never run in newer spark version), can we do it this way:
   Check whether state metadata file exists,
   if metadata exists, read the numColsPrefixKey from the metadata
   if metadata doesn't exist, we use 0 as the numColsPrefixKey and print a warning that this doesn't work for session window operator. Then let runtime exception throw if it is session window operator.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1377043635


##########
sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala:
##########
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf
  * @since 2.0.0
  */
 @Stable
-class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) {
+class RuntimeConfig private[sql](val sqlConf: SQLConf = new SQLConf) {

Review Comment:
   This is probably the smallest change building a new StateStoreConf via copying SQL conf & applying offset metadata configs.
   
   ```
           val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
           OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
           StateStoreConf(clonedRuntimeConf.sqlConf)
   ```
   
   OffsetSeqMetadata.setSessionConf requires RuntimeConfig, the constructor of StateStoreConf requires SQLConf.



##########
sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala:
##########
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf
  * @since 2.0.0
  */
 @Stable
-class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) {
+class RuntimeConfig private[sql](val sqlConf: SQLConf = new SQLConf) {

Review Comment:
   Yes, the above change is about connect compatibility. I don't think Spark connect will leverage this anyway.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/** An implementation of [[Table]] with [[SupportsRead]] for State Store data source. */
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new IllegalStateException(s"Invalid schema is provided. Provided schema: $schema for " +
+      s"checkpoint location: ${sourceOptions.stateCheckpointLocation} , operatorId: " +
+      s"${sourceOptions.operatorId} , storeName: ${sourceOptions.storeName}, " +
+      s"joinSide: ${sourceOptions.joinSide}")
+  }
+
+  override def name(): String = {
+    val desc = s"StateTable " +
+      s"[stateCkptLocation=${sourceOptions.stateCheckpointLocation}]" +
+      s"[batchId=${sourceOptions.batchId}][operatorId=${sourceOptions.operatorId}]" +
+      s"[storeName=${sourceOptions.storeName}]"
+
+    if (sourceOptions.joinSide != JoinSideValues.none) {
+      desc + s"[joinSide=${sourceOptions.joinSide}]"
+    } else {
+      desc
+    }
+  }
+
+  override def capabilities(): util.Set[TableCapability] = CAPABILITY
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+    new StateScanBuilder(session, schema, sourceOptions, stateConf)
+
+  override def properties(): util.Map[String, String] = Map.empty[String, String].asJava
+
+  private def isValidSchema(schema: StructType): Boolean = {
+    if (schema.fieldNames.toSeq != Seq("key", "value")) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "key").isInstanceOf[StructType]) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "value").isInstanceOf[StructType]) {
+      false
+    } else {
+      true
+    }
+  }
+
+  override def metadataColumns(): Array[MetadataColumn] = METADATA_COLUMNS.toArray
+}
+
+object StateTable {
+  private val CAPABILITY = Set(TableCapability.BATCH_READ).asJava

Review Comment:
   Maybe not, as we may consider adding batch write into capability sooner.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID} cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID} cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))

Review Comment:
   No. Do we want to take care of empty string? I don't expect users to specify this except the case of looking into a single state store for stream-stream join. For others they don't need to specify this at all.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)

Review Comment:
   The state schema file is determined by almost all options, path, operatorId, storeName, joinSide. Only batch id is irrelevant. I can enumerate but probably there may be no huge difference.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364983171


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"

Review Comment:
   While I agree that we want to mark this as experimental, I'm a bit unsure that we want to do this via naming change. Maybe the best bet is to document it? And maybe some warning log as well.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364331798


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this
+
+  // FIXME: show more configs?

Review Comment:
   Which other configs are you referring to ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this
+
+  // FIXME: show more configs?
+  override def description(): String = s"StateScan " +
+    s"[stateCkptLocation=$stateCheckpointRootLocation]" +
+    s"[batchId=$batchId][operatorId=$operatorId][storeName=$storeName]" +
+    s"[joinSide=$joinSide]"

Review Comment:
   maybe log this only if join is involved ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    stateCheckpointLocation: String,
+    batchId: Long,
+    operatorId: Int,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new AnalysisException("The fields of schema should be 'key' and 'value', " +

Review Comment:
   Could we also log checkpointLoc and batchId/operatorId maybe ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala:
##########
@@ -185,6 +187,51 @@ class SymmetricHashJoinStateManager(
     }
   }
 
+  def iterator: Iterator[KeyToValuePair] = {

Review Comment:
   nit: Add function comment ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReaderFactory.scala:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StreamStreamJoinStatePartitionReaderFactory(

Review Comment:
   Any reason to keep it in a separate file or can we combine with file above ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericInternalRow, Literal, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateStoreConf, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+class StreamStreamJoinStatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    userFacingSchema: StructType,
+    stateSchema: StructType) extends PartitionReader[InternalRow] with Logging {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(stateSchema, "key")
+    .asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(stateSchema, "value")
+    .asInstanceOf[StructType]
+
+  private val userFacingValueSchema = SchemaUtil.getSchemaAsDataType(userFacingSchema, "value")
+    .asInstanceOf[StructType]
+
+  private val joinSide: JoinSide = partition.joinSide match {
+    case JoinSideValues.left => LeftSide
+    case JoinSideValues.right => RightSide
+    case JoinSideValues.none =>
+      throw new IllegalStateException("Unexpected join side for stream-stream read!")
+  }
+
+  private val (inputAttributes, formatVersion) = {

Review Comment:
   Can we add some comment explaining what this block is doing ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    stateCheckpointLocation: String,
+    batchId: Long,
+    operatorId: Int,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new AnalysisException("The fields of schema should be 'key' and 'value', " +
+      "and each field should have corresponding fields (they should be a StructType)")
+  }
+
+  override def name(): String =
+    s"state-table-ckpt-$stateCheckpointLocation-batch-$batchId-operator-$operatorId-" +
+      s"store-$storeName-joinside-$joinSide-stateconf-$stateConf"
+
+  override def capabilities(): util.Set[TableCapability] = CAPABILITY
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+    new StateScanBuilder(session, schema, stateCheckpointLocation, batchId, operatorId, storeName,
+      joinSide, stateConf)
+
+  // FIXME: pop more critical configurations from stateConf?

Review Comment:
   Can you expand on this a bit ? we want to remove some settings from storeConf ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.JoinSide
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+
+

Review Comment:
   Nit: extra newline ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.JoinSide
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+
+
+object StreamStreamJoinStateHelper {
+  def readSchema(
+      session: SparkSession,
+      stateCheckpointLocation: String,
+      operatorId: Int,
+      side: JoinSide,
+      excludeAuxColumns: Boolean = true): StructType = {
+    val (keySchema, valueSchema) = readKeyValueSchema(session, stateCheckpointLocation,
+      operatorId, side, excludeAuxColumns)
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  def readKeyValueSchema(
+      session: SparkSession,
+      stateCheckpointLocation: String,
+      operatorId: Int,
+      side: JoinSide,
+      excludeAuxColumns: Boolean = true): (StructType, StructType) = {
+
+    // KeyToNumValuesType, KeyWithIndexToValueType
+    val storeNames = SymmetricHashJoinStateManager.allStateStoreNames(side).toList
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeIdForKeyToNumValues = new StateStoreId(stateCheckpointLocation, operatorId,
+      partitionId, storeNames(0))
+    val providerIdForKeyToNumValues = new StateStoreProviderId(storeIdForKeyToNumValues,
+      UUID.randomUUID())
+
+    val storeIdForKeyWithIndexToValue = new StateStoreId(stateCheckpointLocation,
+      operatorId, partitionId, storeNames(1))
+    val providerIdForKeyWithIndexToValue = new StateStoreProviderId(storeIdForKeyWithIndexToValue,
+      UUID.randomUUID())
+
+    val newHadoopConf = session.sessionState.newHadoopConf()
+
+    val manager = new StateSchemaCompatibilityChecker(providerIdForKeyToNumValues, newHadoopConf)
+    val (keySchema, _) = manager.readSchemaFile()
+
+    val manager2 = new StateSchemaCompatibilityChecker(providerIdForKeyWithIndexToValue,
+      newHadoopConf)
+    val (_, valueSchema) = manager2.readSchemaFile()
+
+    val maybeMatchedColumn = valueSchema.last
+
+    if (excludeAuxColumns

Review Comment:
   Which auxiliary columns are we referring to here ? Can you give an example maybe ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1769118727

   @HeartSaVioR - also test failures seem related ?
   
   ```
   HDFSBackedStateDataSourceV2ReadSuite.simple aggregation, state ver 1
   org.apache.spark.SparkClassNotFoundException: [DATA_SOURCE_NOT_FOUND] Failed to find the data source: preview-statestore. Please find packages at `https://spark.apache.org/third-party-projects.html`.
   ```


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1365035655


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericInternalRow, Literal, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateStoreConf, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+class StreamStreamJoinStatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    userFacingSchema: StructType,
+    stateSchema: StructType) extends PartitionReader[InternalRow] with Logging {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(stateSchema, "key")
+    .asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(stateSchema, "value")
+    .asInstanceOf[StructType]
+
+  private val userFacingValueSchema = SchemaUtil.getSchemaAsDataType(userFacingSchema, "value")
+    .asInstanceOf[StructType]
+
+  private val joinSide: JoinSide = partition.joinSide match {
+    case JoinSideValues.left => LeftSide
+    case JoinSideValues.right => RightSide
+    case JoinSideValues.none =>
+      throw new IllegalStateException("Unexpected join side for stream-stream read!")
+  }
+
+  private val (inputAttributes, formatVersion) = {
+    val maybeMatchedColumn = valueSchema.last
+    val (fields, version) = {
+      if (maybeMatchedColumn.name == "matched" && maybeMatchedColumn.dataType == BooleanType) {
+        (valueSchema.dropRight(1), 2)
+      } else {
+        (valueSchema, 1)
+      }
+    }
+
+    assert(fields.toArray.sameElements(userFacingValueSchema.fields),
+      "Exposed fields should be same with given user facing schema for value! " +
+        s"Exposed fields: ${fields.mkString("(", ", ", ")")} / " +
+        s"User facing value fields: ${userFacingValueSchema.fields.mkString("(", ", ", ")")}")
+
+    val attrs = fields.map {
+      f => AttributeReference(f.name, f.dataType, f.nullable)()
+    }
+    (attrs, version)
+  }
+
+  private var joinStateManager: SymmetricHashJoinStateManager = _
+
+  private lazy val iter = {
+    if (joinStateManager == null) {
+      val stateInfo = StatefulOperatorStateInfo(
+        partition.stateCheckpointRootLocation, partition.queryId, partition.operatorId,
+        partition.batchId + 1, -1)
+      joinStateManager = new SymmetricHashJoinStateManager(
+        joinSide,
+        inputAttributes,
+        joinKeys = DataTypeUtils.toAttributes(keySchema),
+        stateInfo = Some(stateInfo),
+        storeConf = storeConf,
+        hadoopConf = hadoopConf.value,
+        partitionId = partition.partition,
+        formatVersion,
+        skippedNullValueCount = None
+      )
+    }
+
+    // state format 2
+    val valueWithMatchedExprs = inputAttributes :+ Literal(true)
+    val indexOrdinalInValueWithMatchedRow = inputAttributes.size
+    val valueWithMatchedRowGenerator = UnsafeProjection.create(valueWithMatchedExprs,
+      inputAttributes)
+
+    joinStateManager.iterator.map { pair =>

Review Comment:
   self-review: need to handle metadata column



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1377056253


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID} cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID} cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))

Review Comment:
   Let's just throw an exception for empty string.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388820123


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala:
##########
@@ -88,8 +88,7 @@ class StateSchemaCompatibilityChecker(
   private def schemasCompatible(storedSchema: StructType, schema: StructType): Boolean =
     DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema)
 
-  // Visible for testing
-  private[sql] def readSchemaFile(): (StructType, StructType) = {

Review Comment:
   It's not about scoping. It is more about the representation that we are exposing these methods to others. check method can be private[sql] as well but we don't, because the method is the entry point. We are adding more entry points, hence making them be public as well.
   (Although someone might argue that it's better to separate the methods for handling reading and writing schema file out to the separate class.)
   
   Technically saying, we should be able to make the entire class be package private, but we don't, because the package is considered as non-public one.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388796953


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"

Review Comment:
   Let's keep this as it is, as we propose this in SPIP doc and it got passed.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1380729958


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +

Review Comment:
   Nit: `Failed to read the`



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,779 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.SparkUnsupportedOperationException
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: trying to read state data as stream") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      intercept[SparkUnsupportedOperationException] {
+        spark.readStream.format("statestore").load(tempDir.getAbsolutePath)
+          .writeStream.format("noop").start()
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "false")
+  }
+}
+
+class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "true")
+  }
+}
+
+abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Assertions {

Review Comment:
   Maybe just add a 1 line class comment ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388798323


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"

Review Comment:
   This is actually for better UX, you pass the checkpoint path in load() and it works smoothly.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1773730435

   MiMa error was a real one - I fixed it. Other tests are not clear - I'm rebasing to latest master branch.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1780463234

   Follow-up ticket:
   https://issues.apache.org/jira/browse/SPARK-45671 to handle corrupt records via flag.
   https://issues.apache.org/jira/browse/SPARK-45672 to provide unified output schema (among state format versions) per operator.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364307075


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new AnalysisException(s"The offset log for $batchId does not exist")

Review Comment:
   Same here ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new AnalysisException(s"The offset log for $batchId does not exist")
+    }
+  }
+
+  private def getLastCommittedBatch(checkpointLocation: String): Long = {
+    val commitLog = new CommitLog(session, new Path(checkpointLocation, "commits").toString)

Review Comment:
   Can we just do
   
   ```
   val commitLog = new CommitLog(session, checkpointLocation + "/commits")
   ```
   ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]

Review Comment:
   Hmm - so we are casting sql DataType to StructType ? will that always be safe ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(

Review Comment:
   Class/file level comment maybe ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new AnalysisException(s"The offset log for $batchId does not exist")
+    }
+  }
+
+  private def getLastCommittedBatch(checkpointLocation: String): Long = {
+    val commitLog = new CommitLog(session, new Path(checkpointLocation, "commits").toString)
+    commitLog.getLatest() match {
+      case Some((lastId, _)) => lastId
+      case None => throw new AnalysisException("No committed batch found.")

Review Comment:
   Can we log more information such as checkpointLoc ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   Currently will this throw an exception/fail the query if we are reading state for a session window query ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364342065


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)

Review Comment:
   Hmm - why do we need to do this ? same question for above



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {

Review Comment:
   Can we add some test cases for failure conditions too ?
   
   - not a stateful query
   - schema is invalid
   - options passed are invalid
   - etc
   ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {

Review Comment:
   Can we just have some tests running with combination of some shuffle partition values and the available codecs ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")

Review Comment:
   nit: intentional ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  test("metadata column") {
+    withTempDir { tempDir =>
+      import testImplicits._
+      val stream = MemoryStream[Int]
+
+      val df = stream.toDF()
+        .groupBy("value")
+        .count()
+
+      stream.addData(1 to 10000: _*)
+
+      val query = df.writeStream.format("noop")
+        .option("checkpointLocation", tempDir.getAbsolutePath)
+        .outputMode(OutputMode.Update())
+        .start()
+
+      query.processAllAvailable()
+      query.stop()
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      assert(!stateReadDf.schema.exists(_.name == "_partition_id"),

Review Comment:
   What is the reason for not exposing this ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {

Review Comment:
   We seem to have bunch of callers to this function in a few places right ?
   
   ```
     private val keySchema = SchemaUtil.getSchemaAsDataType(stateSchema, "key")
       .asInstanceOf[StructType]
     private val valueSchema = SchemaUtil.getSchemaAsDataType(stateSchema, "value")
       .asInstanceOf[StructType]
   
     private val userFacingValueSchema = SchemaUtil.getSchemaAsDataType(userFacingSchema, "value")
       .asInstanceOf[StructType]
   ```
   
   I wonder whether we need to test for the schema file here being invalid/corrupt is throwing an exception as expected ? just some negative cases, in case you think they are worth it/necessary ? if not, we can skip



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {

Review Comment:
   Should we also run this with various output modes ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",

Review Comment:
   How does the user know this format is the one they need use within the select expr ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1769113929

   @HeartSaVioR - some high level questions
   
   - it seems like we only expose partition Id as a metadata/internal column and that too only if its queried ? We don't seem to expose other cols such as batchId/operatorId etc. What is the reason for doing this ?
   - for some of the queries such as join/FMGWS, it seems that we have different formats for v1/v2 and the user needs to query it differently within the selectExpr. How does the user discover these fields ? Is it possible to keep the source schema homogenous here ?
   - for join queries, what schema do we expose when a store name is explicitly specified vs not. I guess the ability to query a specific store name (esp the ones like right-keyToNumValues) is only really for debugging purposes in this case ? Also, for join queries, where do we add the internal metadata cols like partitionId - not sure I found that
   - for the tests, not sure I saw a simulation for the expected use-cases. for eg - some tests where we keep the streaming query running for a few batches and assert for certain conditions/state values along the way. Also, maybe around corruption detection where we artificially corrupt some values and show how the state reader can detect those ?
   - For tests, should we also add some cases with additional startStream/stopStream clauses and verify that state read is working as expected even when batch recovery/restart cases are involved ?
   
   Thanks !


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364994394


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")

Review Comment:
   This makes me thinking.
   
   I expect the query to be used as "interactive", not something they run the query and get the result after hours. That said, they are expected to know the context what they provide in source option.
   
   But for us, when we come to debug this state reader, we may just want to know the checkpoint path rather than figuring out checkpoint path based on these source options.
   
   Maybe this error can be identified as a sort of internal error, and with that we can be less user friendly and more friendly to us. Adding checkpoint path would be better.
   
   Additional logging might be also fine.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376953649


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+}
+
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+  // Here we build a combination of test criteria for
+  // 1) number of shuffle partitions
+  // 2) state store provider
+  // 3) compression codec
+  // and run one of the test to verify that above configs work.
+  // We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+  // and resource to run all combinations for all times, hence we will randomly pick 5 tests
+  // per run.
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,

Review Comment:
   Should we also add a couple of tests with changelog checkpointing enabled ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376943187


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/** An implementation of [[Table]] with [[SupportsRead]] for State Store data source. */
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new IllegalStateException(s"Invalid schema is provided. Provided schema: $schema for " +
+      s"checkpoint location: ${sourceOptions.stateCheckpointLocation} , operatorId: " +
+      s"${sourceOptions.operatorId} , storeName: ${sourceOptions.storeName}, " +
+      s"joinSide: ${sourceOptions.joinSide}")
+  }
+
+  override def name(): String = {
+    val desc = s"StateTable " +
+      s"[stateCkptLocation=${sourceOptions.stateCheckpointLocation}]" +
+      s"[batchId=${sourceOptions.batchId}][operatorId=${sourceOptions.operatorId}]" +
+      s"[storeName=${sourceOptions.storeName}]"
+
+    if (sourceOptions.joinSide != JoinSideValues.none) {
+      desc + s"[joinSide=${sourceOptions.joinSide}]"
+    } else {
+      desc
+    }
+  }
+
+  override def capabilities(): util.Set[TableCapability] = CAPABILITY
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+    new StateScanBuilder(session, schema, sourceOptions, stateConf)
+
+  override def properties(): util.Map[String, String] = Map.empty[String, String].asJava
+
+  private def isValidSchema(schema: StructType): Boolean = {
+    if (schema.fieldNames.toSeq != Seq("key", "value")) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "key").isInstanceOf[StructType]) {
+      false
+    } else if (!SchemaUtil.getSchemaAsDataType(schema, "value").isInstanceOf[StructType]) {
+      false
+    } else {
+      true
+    }
+  }
+
+  override def metadataColumns(): Array[MetadataColumn] = METADATA_COLUMNS.toArray
+}
+
+object StateTable {
+  private val CAPABILITY = Set(TableCapability.BATCH_READ).asJava

Review Comment:
   Can we rename as `BATCH_READ_CAPABILITY` maybe ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1790056322

   Fixed test, added new tests as commented. @anishshri-db Thanks!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1380079245


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   Yeah maybe the better guidance would be encouraging users to run the query with newer Spark version to write the state metadata. There could be some reason which they cannot migrate the workload but want to query the state, but at least we can give the guidance that the limitation will be solved when they migrate.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388807934


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,779 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.SparkUnsupportedOperationException
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: trying to read state data as stream") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      intercept[SparkUnsupportedOperationException] {
+        spark.readStream.format("statestore").load(tempDir.getAbsolutePath)
+          .writeStream.format("noop").start()
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)

Review Comment:
   Do you mean by using withSQLConf? I feel like this is simpler approach if we want to run the entire test cases for various environments.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388804304


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.{JoinSideValues, StateSourceOptions}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+/** An implementation of [[ScanBuilder]] for State Store data source. */
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    sourceOptions: StateSourceOptions,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {

Review Comment:
   Based to the state store API spec, we cannot get any benefit from SupportsPushDownRequiredColumns and SupportsPushDownFilters, except the one, filter by partition ID which we only need to spin a state store for the specific partition. We can file a JIRA ticket as a follow-up.
   
   SupportsPushDownLimit could be something we can consider, though current state store implementations imply that iterator should have been fully consumed to close properly. Let's file a JIRA ticket as a follow-up anyway.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #43425: [SPARK-45511][SS] State Data Source - Reader
URL: https://github.com/apache/spark/pull/43425


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1780421292

   SPIP vote got passed - removed the prefix. It is now ready to be reviewed and merged.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1805890217

   Here are follow-up JIRA tickets:
   
   > MUST (Must be done before releasing this feature, technically before Spark 4.0.0)
   
   * [SPARK-45888](https://issues.apache.org/jira/browse/SPARK-45888) Apply error class framework to state data source & state metadata data source
   * [SPARK-45833](https://issues.apache.org/jira/browse/SPARK-45833) Document state data source - reader
   
   > GOOD TO HAVE (Improvements)
   
   * [SPARK-45671](https://issues.apache.org/jira/browse/SPARK-45671) Implement an option similar to corrupt record column in State Data Source Reader
   * [SPARK-45672](https://issues.apache.org/jira/browse/SPARK-45672) Provide a unified user-facing schema for state format versions in state data source - reader
   * [SPARK-45889](https://issues.apache.org/jira/browse/SPARK-45889) Implement push-down filter with partition ID and grouping key (if possible) for state data source reader
   * [SPARK-45890](https://issues.apache.org/jira/browse/SPARK-45890) Implement limit push down for state data source reader
   
   The PR has been open for 3 weeks and the code change is already 2000+ lines, so I'd prefer to have follow-up works. It would be also a good chance for community to collaborate as well.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388797617


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")

Review Comment:
   I'd love to, but I don't want to be blocked by that. It would require error class framework experts to review and sign-off. Shall we file a "blocker" JIRA ticket to adopt error class framework, and leave this as it is 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "rangadi (via GitHub)" <gi...@apache.org>.
rangadi commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1380760045


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(

Review Comment:
   There is a comment, but it does not give any details about what it does. We could have some information useful for future readers of this code (and current reviewers :)). 



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1378010285


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,695 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "false")
+  }
+}
+
+class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "true")
+  }
+}
+
+abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSource.PARAM_BATCH_ID, batchId)
+          .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          // skip version and operator ID to test out functionalities
+          .load()
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicates with column specified") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQueryWithColumnSpecified(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key.col1 AS key_col1")
+
+      checkAnswer(resultDf, Seq(Row("A"), Row("B"), Row("C"), Row("D")))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.col1 AS key_col1")
+
+      checkAnswer(resultDf2, Seq(Row("A"), Row("B"), Row("C")))
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("metadata column") {
+    withTempDir { tempDir =>
+      import testImplicits._
+      val stream = MemoryStream[Int]
+
+      val df = stream.toDF()
+        .groupBy("value")
+        .count()
+
+      stream.addData(1 to 10000: _*)
+
+      val query = df.writeStream.format("noop")
+        .option("checkpointLocation", tempDir.getAbsolutePath)
+        .outputMode(OutputMode.Update())
+        .start()
+
+      query.processAllAvailable()
+      query.stop()
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      assert(!stateReadDf.schema.exists(_.name == "_partition_id"),
+      "metadata column should not be exposed until it is explicitly specified!")
+
+      val numShufflePartitions = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS)
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "value.count AS value_count", "_partition_id")

Review Comment:
   Should we add a test for join too ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1787661104

   Also another question - do we need to explicit block using this source as part of `readStream` somewhere ? Do we also need a test for this ?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364990355


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)

Review Comment:
   I expect in most cases they want to perform a full scan, or filter "by key". Filtering with partition ID is something only few people needs - maybe only us on investigation on incident. 
   
   Rather than scoping the partition via source option, we could try adding filter pushdown against partition ID for optimization. Let's deal with that functionality into separate ticket.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)

Review Comment:
   Yeah I'll come up with a model for source option. Good point.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)

Review Comment:
   This is basically following what we do in StreamExecution:
   
   ```
     protected def checkpointFile(name: String): String =
       new Path(new Path(resolvedCheckpointRoot), name).toString
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")

Review Comment:
   This makes me thinking.
   
   I expect the query to be used as "interactive", not something they run the query and get the result after hours. That said, they are expected to know the context what they provide in source option.
   
   Also, future DLT support makes thing be more complicated. They no longer know about checkpoint path. They only know about pipeline ID and flow name/ID, so checkpoint path wouldn't help to them.
   
   But for us, when we come to debug this state reader, we may just want to know the checkpoint path rather than figuring out checkpoint path based on these source options.
   
   Maybe this error can be identified as a sort of internal error, and with that we can be less user friendly and more friendly to us. Adding checkpoint path would be better.
   
   Additional logging might be also fine.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   For HDFS backed state store provider, it may just work as there is no difference with underlying format for prefix scan. For RocksDB state store provider, it could throw cryptic exception in runtime.
   
   We are planning to address this, with leveraging operator metadata information. Will file a ticket.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,

Review Comment:
   Yeah I'll think of having a model. Looks like we pass over the same fields again and again among multiple classes.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]

Review Comment:
   Yeah maybe better to distinguish the case from the valid case where there is really no data. Probably better to have a test running this against stateless query and verify the expectation.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this

Review Comment:
   So the interfaces are structured and it is actually burdensome if we create a class per interface. As a workaround this kind of pattern is being used, implementing Scan and Batch altogether in the same class.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2TestBase.scala:
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.{DataFrame, Dataset}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.streaming._
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+trait StateDataSourceV2TestBase extends StreamTest with StateStoreMetricsTest {
+  import testImplicits._
+
+  protected def runCompositeKeyStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getCompositeKeyStreamingAggregationQuery(inputData)
+
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 to 5: _*),
+      CheckLastBatch(
+        (0, "Apple", 1, 0, 0, 0),
+        (1, "Banana", 1, 1, 1, 1),
+        (0, "Strawberry", 1, 2, 2, 2),
+        (1, "Apple", 1, 3, 3, 3),
+        (0, "Banana", 1, 4, 4, 4),
+        (1, "Strawberry", 1, 5, 5, 5)
+      ),
+      // batch 1
+      AddData(inputData, 6 to 10: _*),
+      // state also contains (1, "Strawberry", 1, 5, 5, 5) but not updated here
+      CheckLastBatch(
+        (0, "Apple", 2, 6, 6, 0), // 0, 6
+        (1, "Banana", 2, 8, 7, 1), // 1, 7
+        (0, "Strawberry", 2, 10, 8, 2), // 2, 8
+        (1, "Apple", 2, 12, 9, 3), // 3, 9
+        (0, "Banana", 2, 14, 10, 4) // 4, 10
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 3, 2, 1),
+      CheckLastBatch(
+        (1, "Banana", 3, 9, 7, 1), // 1, 7, 1
+        (0, "Strawberry", 3, 12, 8, 2), // 2, 8, 2
+        (1, "Apple", 3, 15, 9, 3) // 3, 9, 3
+      )
+    )
+  }
+
+  protected def getCompositeKeyStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, String, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 2 AS groupKey",
+        "(CASE value % 3 WHEN 0 THEN 'Apple' WHEN 1 THEN 'Banana' ELSE 'Strawberry' END) AS fruit")
+      .groupBy($"groupKey", $"fruit")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, String, Long, Long, Int, Int)]
+  }
+
+  protected def runLargeDataStreamingAggregationQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val aggregated = getLargeDataStreamingAggregationQuery(inputData)
+
+    // check with more data - leverage full partitions
+    testStream(aggregated, OutputMode.Update)(
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 0
+      AddData(inputData, 0 until 20: _*),
+      CheckLastBatch(
+        (0, 2, 10, 10, 0), // 0, 10
+        (1, 2, 12, 11, 1), // 1, 11
+        (2, 2, 14, 12, 2), // 2, 12
+        (3, 2, 16, 13, 3), // 3, 13
+        (4, 2, 18, 14, 4), // 4, 14
+        (5, 2, 20, 15, 5), // 5, 15
+        (6, 2, 22, 16, 6), // 6, 16
+        (7, 2, 24, 17, 7), // 7, 17
+        (8, 2, 26, 18, 8), // 8, 18
+        (9, 2, 28, 19, 9) // 9, 19
+      ),
+      // batch 1
+      AddData(inputData, 20 until 40: _*),
+      CheckLastBatch(
+        (0, 4, 60, 30, 0), // 0, 10, 20, 30
+        (1, 4, 64, 31, 1), // 1, 11, 21, 31
+        (2, 4, 68, 32, 2), // 2, 12, 22, 32
+        (3, 4, 72, 33, 3), // 3, 13, 23, 33
+        (4, 4, 76, 34, 4), // 4, 14, 24, 34
+        (5, 4, 80, 35, 5), // 5, 15, 25, 35
+        (6, 4, 84, 36, 6), // 6, 16, 26, 36
+        (7, 4, 88, 37, 7), // 7, 17, 27, 37
+        (8, 4, 92, 38, 8), // 8, 18, 28, 38
+        (9, 4, 96, 39, 9) // 9, 19, 29, 39
+      ),
+      StopStream,
+      StartStream(checkpointLocation = checkpointRoot),
+      // batch 2
+      AddData(inputData, 0, 1, 2),
+      CheckLastBatch(
+        (0, 5, 60, 30, 0), // 0, 10, 20, 30, 0
+        (1, 5, 65, 31, 1), // 1, 11, 21, 31, 1
+        (2, 5, 70, 32, 2) // 2, 12, 22, 32, 2
+      )
+    )
+  }
+
+  protected def getLargeDataStreamingAggregationQuery(
+      inputData: MemoryStream[Int]): Dataset[(Int, Long, Long, Int, Int)] = {
+    inputData.toDF()
+      .selectExpr("value", "value % 10 AS groupKey")
+      .groupBy($"groupKey")
+      .agg(
+        count("*").as("cnt"),
+        sum("value").as("sum"),
+        max("value").as("max"),
+        min("value").as("min")
+      )
+      .as[(Int, Long, Long, Int, Int)]
+  }
+
+  protected def runDropDuplicatesQuery(checkpointRoot: String): Unit = {
+    val inputData = MemoryStream[Int]
+    val deduplicated = getDropDuplicatesQuery(inputData)
+
+    testStream(deduplicated, OutputMode.Append())(
+      StartStream(checkpointLocation = checkpointRoot),
+
+      AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*),
+      CheckAnswer(10 to 15: _*),
+      assertNumStateRows(total = 6, updated = 6),
+
+      AddData(inputData, 25), // Advance watermark to 15 secs, no-data-batch drops rows <= 15
+      CheckNewAnswer(25),
+      assertNumStateRows(total = 1, updated = 1),
+
+      AddData(inputData, 10), // Should not emit anything as data less than watermark
+      CheckNewAnswer(),
+      assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1),
+
+      AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25
+      CheckNewAnswer(45),
+      assertNumStateRows(total = 1, updated = 1)
+    )
+  }
+
+  protected def getDropDuplicatesQuery(inputData: MemoryStream[Int]): Dataset[Long] = {

Review Comment:
   Will check and descope if feasible.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateTable(
+    session: SparkSession,
+    override val schema: StructType,
+    stateCheckpointLocation: String,
+    batchId: Long,
+    operatorId: Int,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateConf: StateStoreConf)
+  extends Table with SupportsRead with SupportsMetadataColumns {
+
+  import StateTable._
+
+  if (!isValidSchema(schema)) {
+    throw new AnalysisException("The fields of schema should be 'key' and 'value', " +
+      "and each field should have corresponding fields (they should be a StructType)")
+  }
+
+  override def name(): String =
+    s"state-table-ckpt-$stateCheckpointLocation-batch-$batchId-operator-$operatorId-" +
+      s"store-$storeName-joinside-$joinSide-stateconf-$stateConf"
+
+  override def capabilities(): util.Set[TableCapability] = CAPABILITY
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
+    new StateScanBuilder(session, schema, stateCheckpointLocation, batchId, operatorId, storeName,
+      joinSide, stateConf)
+
+  // FIXME: pop more critical configurations from stateConf?

Review Comment:
   I revisited this and now inclined that this is probably not needed. Haven't realized that it is most likely for SHOW TBLPROPERTIES which this data source won't support anyway.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {

Review Comment:
   It's not realistic to create test for every combination. We know the output mode does not make a difference for the state "format", hence may want to skip creating a full combination.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {

Review Comment:
   Let's see whether it triggers too much test duration being added or not. It seems like the functionality is/should be tested against state store provider itself, but we can revisit.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]

Review Comment:
   Yes, because we validate the type of schema in StateTable.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new AnalysisException(s"The offset log for $batchId does not exist")
+    }
+  }
+
+  private def getLastCommittedBatch(checkpointLocation: String): Long = {
+    val commitLog = new CommitLog(session, new Path(checkpointLocation, "commits").toString)

Review Comment:
   Same as above. current code fits to the best practice.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this
+
+  // FIXME: show more configs?
+  override def description(): String = s"StateScan " +
+    s"[stateCkptLocation=$stateCheckpointRootLocation]" +
+    s"[batchId=$batchId][operatorId=$operatorId][storeName=$storeName]" +
+    s"[joinSide=$joinSide]"

Review Comment:
   You meant when joinSide option is specified, right? then yeah makes sense.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()

Review Comment:
   While it should just be dummy query ID, it's easier to identify the partitions for each attempt via dummy query ID.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an
+    //  information whether the state store uses prefix scan or not. We will have to add such
+    //  information to determine the right encoder/decoder for the data.
+    StateStore.getReadOnly(stateStoreProviderId, keySchema, valueSchema,
+      numColsPrefixKey = 0, version = partition.batchId + 1, storeConf = storeConf,
+      hadoopConf = hadoopConf.value)
+  }
+
+  private lazy val iter = {
+    store.iterator().map(pair => unifyStateRowPair((pair.key, pair.value)))
+  }
+
+  private var current: InternalRow = _
+
+  override def next(): Boolean = {
+    if (iter.hasNext) {
+      current = iter.next()
+      true
+    } else {
+      current = null
+      false
+    }
+  }
+
+  private val joinedRow = new JoinedRow()
+
+  private def addMetadata(row: InternalRow): InternalRow = {
+    val metadataRow = new GenericInternalRow(
+      StateTable.METADATA_COLUMNS.map(_.name()).map {
+        case "_partition_id" => partition.partition.asInstanceOf[Any]
+      }.toArray
+    )
+    joinedRow.withLeft(row).withRight(metadataRow)
+  }
+
+  override def get(): InternalRow = addMetadata(current)
+
+  override def close(): Unit = {
+    current = null
+    store.abort()

Review Comment:
   Yes for now. We may have to touch the state store (provider) API to avoid triggering coordination, which I want to put aside from baseline implementation. Maybe we could file a JIRA ticket for this.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)

Review Comment:
   Yeah I forgot that session won't be used across test suite. Will remove.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this
+
+  // FIXME: show more configs?

Review Comment:
   Maybe I can remove the comment now.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {

Review Comment:
   Yeah I'll check and come up with negative cases.
   
   For the case of schema being invalid, there would be mostly no case because
   
   * Users are unable to provide the schema as source option.
   * If there is a state schema file, it is the source of truth. UnsafeRow (and underlying serialized data) does not have an information about schema and we just project the value based on the schema we have in state schema file. If there are mismatch between schema and the row, it would end up with runtime exception, and even worse, JVM level error. That was one of the main reason I broght up the state schema; previously there is no schema information at all and no check with compatibility.
   
   But maybe I can revisit the case where the schema "file" is corrupted. 



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")

Review Comment:
   No, we can remove it.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new AnalysisException(s"The offset log for $batchId does not exist")

Review Comment:
   Yeah same thought applies. Adding checkpoint path would be good.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.JoinSide
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.types.{BooleanType, StructType}
+
+
+object StreamStreamJoinStateHelper {
+  def readSchema(
+      session: SparkSession,
+      stateCheckpointLocation: String,
+      operatorId: Int,
+      side: JoinSide,
+      excludeAuxColumns: Boolean = true): StructType = {
+    val (keySchema, valueSchema) = readKeyValueSchema(session, stateCheckpointLocation,
+      operatorId, side, excludeAuxColumns)
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  def readKeyValueSchema(
+      session: SparkSession,
+      stateCheckpointLocation: String,
+      operatorId: Int,
+      side: JoinSide,
+      excludeAuxColumns: Boolean = true): (StructType, StructType) = {
+
+    // KeyToNumValuesType, KeyWithIndexToValueType
+    val storeNames = SymmetricHashJoinStateManager.allStateStoreNames(side).toList
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeIdForKeyToNumValues = new StateStoreId(stateCheckpointLocation, operatorId,
+      partitionId, storeNames(0))
+    val providerIdForKeyToNumValues = new StateStoreProviderId(storeIdForKeyToNumValues,
+      UUID.randomUUID())
+
+    val storeIdForKeyWithIndexToValue = new StateStoreId(stateCheckpointLocation,
+      operatorId, partitionId, storeNames(1))
+    val providerIdForKeyWithIndexToValue = new StateStoreProviderId(storeIdForKeyWithIndexToValue,
+      UUID.randomUUID())
+
+    val newHadoopConf = session.sessionState.newHadoopConf()
+
+    val manager = new StateSchemaCompatibilityChecker(providerIdForKeyToNumValues, newHadoopConf)
+    val (keySchema, _) = manager.readSchemaFile()
+
+    val manager2 = new StateSchemaCompatibilityChecker(providerIdForKeyWithIndexToValue,
+      newHadoopConf)
+    val (_, valueSchema) = manager2.readSchemaFile()
+
+    val maybeMatchedColumn = valueSchema.last
+
+    if (excludeAuxColumns

Review Comment:
   It represents internal columns which are added from stream-stream join operators. As of now, "matched" is the only column the operator is adding.
   
   I'll add a code comment. Maybe better to add a simple method doc.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",

Review Comment:
   They can print the schema in prior before running the query, like stateReadDf.schema.treeString or printSchema().



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2ReadSuite.scala:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.scalatest.Assertions
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class HDFSBackedStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+class RocksDBStateDataSourceV2ReadSuite extends StateDataSourceV2ReadSuite {
+  private var oldProviderClass: String = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+
+    oldProviderClass = spark.conf.get(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+  }
+
+  override def afterAll(): Unit = {
+    if (oldProviderClass != null) {
+      spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, oldProviderClass)
+    } else {
+      spark.conf.unset(SQLConf.STATE_STORE_PROVIDER_CLASS.key)
+    }
+
+    super.afterAll()
+  }
+}
+
+abstract class StateDataSourceV2ReadSuite extends StateDataSourceV2TestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSourceV2.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSourceV2.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("preview-statestore")
+          .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSourceV2.PARAM_BATCH_ID, batchId)
+          .option(StateDataSourceV2.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        logInfo(s"Schema: ${stateReadDf.schema.treeString}")
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  test("metadata column") {
+    withTempDir { tempDir =>
+      import testImplicits._
+      val stream = MemoryStream[Int]
+
+      val df = stream.toDF()
+        .groupBy("value")
+        .count()
+
+      stream.addData(1 to 10000: _*)
+
+      val query = df.writeStream.format("noop")
+        .option("checkpointLocation", tempDir.getAbsolutePath)
+        .outputMode(OutputMode.Update())
+        .start()
+
+      query.processAllAvailable()
+      query.stop()
+
+      val stateReadDf = spark.read
+        .format("preview-statestore")
+        .option(StateDataSourceV2.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      assert(!stateReadDf.schema.exists(_.name == "_partition_id"),

Review Comment:
   We wouldn't like to expose columns which are unrelated to the state row by default. We want to expose these columns as metadata columns, so they only appear when users explicitly select these columns.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1773208724

   @HeartSaVioR - test failures now seem unrelated right ?


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1767714219

   I'll file a new JIRA ticket about documentation and handle it separately once we pass SPIP process. The code change is already about 1.7k.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364319222


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an
+    //  information whether the state store uses prefix scan or not. We will have to add such
+    //  information to determine the right encoder/decoder for the data.
+    StateStore.getReadOnly(stateStoreProviderId, keySchema, valueSchema,
+      numColsPrefixKey = 0, version = partition.batchId + 1, storeConf = storeConf,
+      hadoopConf = hadoopConf.value)
+  }
+
+  private lazy val iter = {
+    store.iterator().map(pair => unifyStateRowPair((pair.key, pair.value)))
+  }
+
+  private var current: InternalRow = _
+
+  override def next(): Boolean = {
+    if (iter.hasNext) {
+      current = iter.next()
+      true
+    } else {
+      current = null
+      false
+    }
+  }
+
+  private val joinedRow = new JoinedRow()
+
+  private def addMetadata(row: InternalRow): InternalRow = {
+    val metadataRow = new GenericInternalRow(
+      StateTable.METADATA_COLUMNS.map(_.name()).map {
+        case "_partition_id" => partition.partition.asInstanceOf[Any]
+      }.toArray
+    )
+    joinedRow.withLeft(row).withRight(metadataRow)
+  }
+
+  override def get(): InternalRow = addMetadata(current)
+
+  override def close(): Unit = {
+    current = null
+    store.abort()

Review Comment:
   Are we relying on coordinator for this change ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]

Review Comment:
   This essentially means this query is not stateful ? Should we throw an exception in this case ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()
+
+      val partitionsSorted = partitions.sortBy(fs => fs.getPath.getName.toInt)
+      val partitionNums = partitionsSorted.map(_.getPath.getName.toInt)
+      // assuming no same number - they're directories hence no same name
+      val head = partitionNums.head
+      val tail = partitionNums(partitionNums.length - 1)
+      assert(head == 0, "Partition should start with 0")
+      assert((tail - head + 1) == partitionNums.length,
+        s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}")
+
+      partitionNums.map {
+        pn => new StateStoreInputPartition(pn, queryId, stateCheckpointRootLocation,
+          batchId, operatorId, storeName, joinSide)
+      }.toArray
+    }
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = joinSide match {
+    case JoinSideValues.left =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, LeftSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.right =>
+      val userFacingSchema = schema
+      val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
+        stateCheckpointRootLocation, operatorId.toInt, RightSide, excludeAuxColumns = false)
+      new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
+        hadoopConfBroadcast.value, userFacingSchema, stateSchema)
+
+    case JoinSideValues.none =>
+      new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema)
+  }
+
+  override def toBatch: Batch = this

Review Comment:
   What does this do exactly ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReaderFactory.scala:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReaderFactory(

Review Comment:
   Can we move this within the StatePartitionReader.scala file itself ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends Scan with Batch {
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val hadoopConfBroadcast = session.sparkContext.broadcast(
+    new SerializableConfiguration(session.sessionState.newHadoopConf()))
+
+  override def readSchema(): StructType = schema
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    val fs = stateCheckpointPartitionsLocation.getFileSystem(hadoopConfBroadcast.value.value)
+    val partitions = fs.listStatus(stateCheckpointPartitionsLocation, new PathFilter() {
+      override def accept(path: Path): Boolean = {
+        fs.isDirectory(path) && Try(path.getName.toInt).isSuccess && path.getName.toInt >= 0
+      }
+    })
+
+    if (partitions.headOption.isEmpty) {
+      Array.empty[InputPartition]
+    } else {
+      // just a dummy query id because we are actually not running streaming query
+      val queryId = UUID.randomUUID()

Review Comment:
   Hmm - why do we need to pass dummy queryId here ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(

Review Comment:
   Can we add some small comments for these classes or maybe just a file level comment ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util.UUID
+
+import scala.util.Try
+
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceV2.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StateScanBuilder(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,
+    operatorId: Long,
+    storeName: String,
+    joinSide: JoinSideValues,
+    stateStoreConf: StateStoreConf) extends ScanBuilder {
+  override def build(): Scan = new StateScan(session, schema, stateCheckpointRootLocation,
+    batchId, operatorId, storeName, joinSide, stateStoreConf)
+}
+
+class StateStoreInputPartition(
+    val partition: Int,
+    val queryId: UUID,
+    val stateCheckpointRootLocation: String,
+    val batchId: Long,
+    val operatorId: Long,
+    val storeName: String,
+    val joinSide: JoinSideValues) extends InputPartition
+
+class StateScan(
+    session: SparkSession,
+    schema: StructType,
+    stateCheckpointRootLocation: String,
+    batchId: Long,

Review Comment:
   Can we embed some of these params within a case class ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1364305033


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)

Review Comment:
   We don't allow filtering for partition id ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for $batchId")

Review Comment:
   Can we log queryId too ? Maybe also checkpoint path ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {

Review Comment:
   Can we add some comments for these higher level classes ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"

Review Comment:
   Should we mark this as `experimental` for the initial release ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)

Review Comment:
   Can we combine into a single function for all these confs ?
   
   Seems like we are doing the same operations above ?
   
   ```
       val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
         throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
       }.get
   
       val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
   
       val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
         Some(getLastCommittedBatch(resolvedCpLocation))
       }.get
   
       val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceV2.scala:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{AnalysisException, RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class StateDataSourceV2 extends TableProvider with DataSourceRegister {
+  import StateDataSourceV2._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val checkpointLocation = Option(properties.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val batchId = Option(properties.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+      Some(getLastCommittedBatch(resolvedCpLocation))
+    }.get
+
+    val operatorId = Option(properties.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val storeName = Option(properties.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(properties.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    val stateConf = buildStateStoreConf(resolvedCpLocation, batchId)
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    new StateTable(session, schema, stateCheckpointLocation.toString, batchId, operatorId,
+      storeName, joinSide, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+      throw new AnalysisException(s"'$PARAM_PATH' must be specified.")
+    }.get
+
+    val resolvedCpLocation = resolvedCheckpointLocation(checkpointLocation)
+
+    val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+      .orElse(Some(0)).get
+
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val storeName = Option(options.get(PARAM_STORE_NAME))
+      .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+    val joinSide = Option(options.get(PARAM_JOIN_SIDE))
+      .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+
+    if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = new Path(resolvedCpLocation, "state")
+    val (keySchema, valueSchema) = joinSide match {
+      case JoinSideValues.left =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, LeftSide)
+
+      case JoinSideValues.right =>
+        StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+          operatorId, RightSide)
+
+      case JoinSideValues.none =>
+        val storeId = new StateStoreId(stateCheckpointLocation.toString, operatorId, partitionId,
+          storeName)
+        val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+        val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+        manager.readSchemaFile()
+    }
+
+    new StructType()
+      .add("key", keySchema)
+      .add("value", valueSchema)
+  }
+
+  private def resolvedCheckpointLocation(checkpointLocation: String): String = {
+    val checkpointPath = new Path(checkpointLocation)
+    val fs = checkpointPath.getFileSystem(hadoopConf)
+    checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)

Review Comment:
   Same here ?
   
   ```
   val offsetLog = new OffsetSeqLog(session, checkpointLocation + "/offsets")
   ```
   



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1378010285


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,695 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: store name is empty") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_STORE_NAME, "")
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+}
+
+/**
+ * Here we build a combination of test criteria for
+ * 1) number of shuffle partitions
+ * 2) state store provider
+ * 3) compression codec
+ * and run one of the test to verify that above configs work.
+ *
+ * We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+ * and resource to run all combinations for all times, hence we will randomly pick 5 tests
+ * per run.
+ */
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "false")
+  }
+}
+
+class RocksDBWithChangelogCheckpointStateDataSourceReaderSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[RocksDBStateStoreProvider].getName)
+    spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled",
+      "true")
+  }
+}
+
+abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Assertions {
+
+  test("simple aggregation, state ver 1") {
+    testStreamingAggregation(1)
+  }
+
+  test("simple aggregation, state ver 2") {
+    testStreamingAggregation(2)
+  }
+
+  test("composite key aggregation, state ver 1") {
+    testStreamingAggregationWithCompositeKey(1)
+  }
+
+  test("composite key aggregation, state ver 2") {
+    testStreamingAggregationWithCompositeKey(2)
+  }
+
+  private def testStreamingAggregation(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val operatorId = 0
+        val batchId = 2
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          // explicitly specifying batch ID and operator ID to test out the functionality
+          .option(StateDataSource.PARAM_BATCH_ID, batchId)
+          .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+          .load()
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+            "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+            Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+            Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+            Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+            Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+            Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+            Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+            Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+            Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+            Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+          )
+        )
+      }
+    }
+  }
+
+  private def testStreamingAggregationWithCompositeKey(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runCompositeKeyStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          // skip version and operator ID to test out functionalities
+          .load()
+
+        val resultDf = stateReadDf
+          .selectExpr("key.groupKey AS key_groupKey", "key.fruit AS key_fruit",
+            "value.count AS value_cnt", "value.sum AS value_sum", "value.max AS value_max",
+            "value.min AS value_min")
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row(0, "Apple", 2, 6, 6, 0),
+            Row(1, "Banana", 3, 9, 7, 1),
+            Row(0, "Strawberry", 3, 12, 8, 2),
+            Row(1, "Apple", 3, 15, 9, 3),
+            Row(0, "Banana", 2, 14, 10, 4),
+            Row(1, "Strawberry", 1, 5, 5, 5)
+          )
+        )
+      }
+    }
+  }
+
+  test("dropDuplicates") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf, Seq(Row(45, 45)))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.value AS key_value", "CAST(key.eventTime AS LONG) AS key_eventTime_long")
+
+      checkAnswer(resultDf2,
+        (10 to 15).map(idx => Row(idx, idx))
+      )
+    }
+  }
+
+  test("dropDuplicates with column specified") {
+    withTempDir { tempDir =>
+      runDropDuplicatesQueryWithColumnSpecified(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key.col1 AS key_col1")
+
+      checkAnswer(resultDf, Seq(Row("A"), Row("B"), Row("C"), Row("D")))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 0)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key.col1 AS key_col1")
+
+      checkAnswer(resultDf2, Seq(Row("A"), Row("B"), Row("C")))
+    }
+  }
+
+  test("dropDuplicatesWithinWatermark") {
+    withTempDir { tempDir =>
+      runDropDuplicatesWithinWatermarkQuery(tempDir.getAbsolutePath)
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      val resultDf = stateReadDf
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf,
+        Seq(Row("b", 24000000), Row("d", 27000000)))
+
+      val stateReadDf2 = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        .option(StateDataSource.PARAM_BATCH_ID, 4)
+        .load()
+
+      val resultDf2 = stateReadDf2
+        .selectExpr("key._1 AS key_1", "value.expiresAtMicros AS value_expiresAtMicros")
+
+      checkAnswer(resultDf2,
+        Seq(
+          Row("a", 19000000),
+          Row("b", 24000000),
+          Row("c", 23000000)
+        )
+      )
+    }
+  }
+
+  test("flatMapGroupsWithState, state ver 1") {
+    testFlatMapGroupsWithState(1)
+  }
+
+  test("flatMapGroupsWithState, state ver 2") {
+    testFlatMapGroupsWithState(2)
+  }
+
+  private def testFlatMapGroupsWithState(stateVersion: Int): Unit = {
+    withSQLConf(SQLConf.FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runFlatMapGroupsWithStateQuery(tempDir.getAbsolutePath)
+
+        val stateReadDf = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .load()
+
+        val resultDf = if (stateVersion == 1) {
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.numEvents AS value_numEvents",
+              "value.startTimestampMs AS value_startTimestampMs",
+              "value.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        } else { // stateVersion == 2
+          stateReadDf
+            .selectExpr("key.value AS key_value", "value.groupState.numEvents AS value_numEvents",
+              "value.groupState.startTimestampMs AS value_startTimestampMs",
+              "value.groupState.endTimestampMs AS value_endTimestampMs",
+              "value.timeoutTimestamp AS value_timeoutTimestamp")
+        }
+
+        checkAnswer(
+          resultDf,
+          Seq(
+            Row("hello", 4, 1000, 4000, 12000),
+            Row("world", 2, 1000, 3000, 12000),
+            Row("scala", 2, 2000, 4000, 12000)
+          )
+        )
+
+        // try to read the value via case class provided in actual query
+        implicit val encoder = Encoders.product[SessionInfo]
+        val df = if (stateVersion == 1) {
+          stateReadDf.selectExpr("value.*").drop("timeoutTimestamp").as[SessionInfo]
+        } else { // state version == 2
+          stateReadDf.selectExpr("value.groupState.*").as[SessionInfo]
+        }
+
+        val expected = Array(
+          SessionInfo(4, 1000, 4000),
+          SessionInfo(2, 1000, 3000),
+          SessionInfo(2, 2000, 4000)
+        )
+        assert(df.collect().toSet === expected.toSet)
+      }
+    }
+  }
+
+  test("stream-stream join, state ver 1") {
+    testStreamStreamJoin(1)
+  }
+
+  test("stream-stream join, state ver 2") {
+    testStreamStreamJoin(2)
+  }
+
+  private def testStreamStreamJoin(stateVersion: Int): Unit = {
+    def assertInternalColumnIsNotExposed(df: DataFrame): Unit = {
+      val valueSchema = SchemaUtil.getSchemaAsDataType(df.schema, "value")
+        .asInstanceOf[StructType]
+
+      intercept[AnalysisException] {
+        SchemaUtil.getSchemaAsDataType(valueSchema, "matched")
+      }
+    }
+
+    withSQLConf(SQLConf.STREAMING_JOIN_STATE_FORMAT_VERSION.key -> stateVersion.toString) {
+      withTempDir { tempDir =>
+        runStreamStreamJoinQuery(tempDir.getAbsolutePath)
+        val stateReaderForLeft = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_JOIN_SIDE, "left")
+
+        val stateReadDfForLeft = stateReaderForLeft.load()
+        assertInternalColumnIsNotExposed(stateReadDfForLeft)
+
+        val resultDf = stateReadDfForLeft
+          .selectExpr("key.field0 As key_0", "value.leftId AS leftId",
+            "CAST(value.leftTime AS integer) AS leftTime")
+
+        checkAnswer(
+          resultDf,
+          Seq(Row(2, 2, 2L), Row(4, 4, 4L), Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRight = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+
+        val stateReadDfForRight = stateReaderForRight.load()
+        assertInternalColumnIsNotExposed(stateReadDfForRight)
+
+        val resultDf2 = stateReadDfForRight
+          .selectExpr("key.field0 As key_0", "value.rightId AS rightId",
+            "CAST(value.rightTime AS integer) AS rightTime")
+
+        checkAnswer(
+          resultDf2,
+          Seq(Row(6, 6, 6L), Row(8, 8, 8L), Row(10, 10, 10L))
+        )
+
+        val stateReaderForRightKeyToNumValues = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_STORE_NAME,
+            "right-keyToNumValues")
+
+        val stateReadDfForRightKeyToNumValues = stateReaderForRightKeyToNumValues.load()
+        val resultDf3 = stateReadDfForRightKeyToNumValues
+          .selectExpr("key.field0 AS key_0", "value.value")
+
+        checkAnswer(
+          resultDf3,
+          Seq(Row(6, 1L), Row(8, 1L), Row(10, 1L))
+        )
+
+        val stateReaderForRightKeyWithIndexToValue = spark.read
+          .format("statestore")
+          .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+          .option(StateDataSource.PARAM_STORE_NAME,
+            "right-keyWithIndexToValue")
+
+        val stateReadDfForRightKeyWithIndexToValue = stateReaderForRightKeyWithIndexToValue.load()
+
+        if (stateVersion == 2) {
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime",
+              "value.matched As matched")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L, true), Row(8, 0, 8, 8L, true), Row(10, 0, 10, 10L, true))
+          )
+        } else {
+          // stateVersion == 1
+          val resultDf4 = stateReadDfForRightKeyWithIndexToValue
+            .selectExpr("key.field0 AS key_0", "key.index AS key_index",
+              "value.rightId AS rightId", "CAST(value.rightTime AS integer) AS rightTime")
+
+          checkAnswer(
+            resultDf4,
+            Seq(Row(6, 0, 6, 6L), Row(8, 0, 8, 8L), Row(10, 0, 10, 10L))
+          )
+        }
+      }
+    }
+  }
+
+  test("metadata column") {
+    withTempDir { tempDir =>
+      import testImplicits._
+      val stream = MemoryStream[Int]
+
+      val df = stream.toDF()
+        .groupBy("value")
+        .count()
+
+      stream.addData(1 to 10000: _*)
+
+      val query = df.writeStream.format("noop")
+        .option("checkpointLocation", tempDir.getAbsolutePath)
+        .outputMode(OutputMode.Update())
+        .start()
+
+      query.processAllAvailable()
+      query.stop()
+
+      val stateReadDf = spark.read
+        .format("statestore")
+        .option(StateDataSource.PARAM_PATH, tempDir.getAbsolutePath)
+        // skip version and operator ID to test out functionalities
+        .load()
+
+      assert(!stateReadDf.schema.exists(_.name == "_partition_id"),
+      "metadata column should not be exposed until it is explicitly specified!")
+
+      val numShufflePartitions = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS)
+
+      val resultDf = stateReadDf
+        .selectExpr("key.value AS key_value", "value.count AS value_count", "_partition_id")

Review Comment:
   Should we add a test for stream-stream join too to test for the internal/metadata column ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376951304


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+}
+
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+  // Here we build a combination of test criteria for

Review Comment:
   Should we move up to class level comment ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1811428536

   I'll rebase to retrigger CI and merge if everything is good.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1811667303

   Thanks all for reviewing! Merging to master.


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1376940262


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)

Review Comment:
   Do we need to log all `sourceOptions` here ?



##########
sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala:
##########
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf
  * @since 2.0.0
  */
 @Stable
-class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) {
+class RuntimeConfig private[sql](val sqlConf: SQLConf = new SQLConf) {

Review Comment:
   Why do we need this change and the one is connect client compatibility above ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Fail to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID} cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID} cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))

Review Comment:
   Will this set store name to default if its provided as empty string ?



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1377059613


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.io.{File, FileWriter}
+
+import org.scalatest.Assertions
+
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row}
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog}
+import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider, StateStore}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.{IntegerType, StructType}
+
+class StateDataSourceNegativeTestSuite extends StateDataSourceTestBase {
+  import testImplicits._
+
+  test("ERROR: read the state from stateless query") {
+    withTempDir { tempDir =>
+      val inputData = MemoryStream[Int]
+      val df = inputData.toDF()
+        .selectExpr("value", "value % 2 AS value2")
+
+      testStream(df)(
+        StartStream(checkpointLocation = tempDir.getAbsolutePath),
+        AddData(inputData, 1, 2, 3, 4, 5),
+        CheckLastBatch((1, 1), (2, 0), (3, 1), (4, 0), (5, 1)),
+        AddData(inputData, 6, 7, 8),
+        CheckLastBatch((6, 0), (7, 1), (8, 0))
+      )
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: no committed batch on default batch ID") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      val offsetLog = new OffsetSeqLog(spark,
+        new File(tempDir.getAbsolutePath, "offsets").getAbsolutePath)
+      val commitLog = new CommitLog(spark,
+        new File(tempDir.getAbsolutePath, "commits").getAbsolutePath)
+
+      offsetLog.purgeAfter(0)
+      commitLog.purgeAfter(-1)
+
+      intercept[IllegalStateException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: corrupted state schema file") {
+    withTempDir { tempDir =>
+      runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+      def rewriteStateSchemaFileToDummy(): Unit = {
+        // Refer to the StateSchemaCompatibilityChecker for the path of state schema file
+        val pathForSchema = Seq(
+          "state", "0", StateStore.PARTITION_ID_TO_CHECK_SCHEMA.toString,
+          "_metadata", "schema"
+        ).foldLeft(tempDir) { case (file, dirName) =>
+          new File(file, dirName)
+        }
+
+        assert(pathForSchema.exists())
+        assert(pathForSchema.delete())
+
+        val fileWriter = new FileWriter(pathForSchema)
+        fileWriter.write("lol dummy corrupted schema file")
+        fileWriter.close()
+
+        assert(pathForSchema.exists())
+      }
+
+      rewriteStateSchemaFileToDummy()
+
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore").load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: path is not specified") {
+    intercept[IllegalArgumentException] {
+      spark.read.format("statestore").load()
+    }
+  }
+
+  test("ERROR: operator ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_OPERATOR_ID, -1)
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: batch ID specified to negative") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_BATCH_ID, -1)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: invalid value for joinSide option") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "both")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  test("ERROR: both options `joinSide` and `storeName` are specified") {
+    withTempDir { tempDir =>
+      intercept[IllegalArgumentException] {
+        spark.read.format("statestore")
+          .option(StateDataSource.PARAM_JOIN_SIDE, "right")
+          .option(StateDataSource.PARAM_STORE_NAME, "right-keyToNumValues")
+          // trick to bypass getting the last committed batch before validating operator ID
+          .option(StateDataSource.PARAM_BATCH_ID, 0)
+          .load(tempDir.getAbsolutePath)
+      }
+    }
+  }
+}
+
+class StateDataSourceSQLConfigSuite extends StateDataSourceTestBase {
+  // Here we build a combination of test criteria for
+  // 1) number of shuffle partitions
+  // 2) state store provider
+  // 3) compression codec
+  // and run one of the test to verify that above configs work.
+  // We are building 3 x 2 x 4 = 24 different test criteria, and it's probably waste of time
+  // and resource to run all combinations for all times, hence we will randomly pick 5 tests
+  // per run.
+
+  private val TEST_SHUFFLE_PARTITIONS = Seq(1, 3, 5)
+  private val TEST_PROVIDERS = Seq(
+    classOf[HDFSBackedStateStoreProvider].getName,
+    classOf[RocksDBStateStoreProvider].getName
+  )
+  private val TEST_COMPRESSION_CODECS = CompressionCodec.ALL_COMPRESSION_CODECS
+
+  private val ALL_COMBINATIONS = {
+    val comb = for (
+      part <- TEST_SHUFFLE_PARTITIONS;
+      provider <- TEST_PROVIDERS;
+      codec <- TEST_COMPRESSION_CODECS
+    ) yield {
+      (part, provider, codec)
+    }
+    scala.util.Random.shuffle(comb)
+  }
+
+  ALL_COMBINATIONS.take(5).foreach { case (part, provider, codec) =>
+    val testName = s"Verify the read with config [part=$part][provider=$provider][codec=$codec]"
+    test(testName) {
+      withTempDir { tempDir =>
+        withSQLConf(
+          SQLConf.SHUFFLE_PARTITIONS.key -> part.toString,
+          SQLConf.STATE_STORE_PROVIDER_CLASS.key -> provider,
+          SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> codec) {
+
+          runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+
+          verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+        }
+      }
+    }
+  }
+
+  test("Use different configs than session config") {
+    withTempDir { tempDir =>
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "3",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "zstd") {
+
+        runLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+
+      // Set the different values in session config, to validate whether state data source refers
+      // to the config in offset log.
+      withSQLConf(
+        SQLConf.SHUFFLE_PARTITIONS.key -> "5",
+        SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[HDFSBackedStateStoreProvider].getName,
+        SQLConf.STATE_STORE_COMPRESSION_CODEC.key -> "lz4") {
+
+        verifyLargeDataStreamingAggregationQuery(tempDir.getAbsolutePath)
+      }
+    }
+  }
+
+  private def verifyLargeDataStreamingAggregationQuery(checkpointLocation: String): Unit = {
+    val operatorId = 0
+    val batchId = 2
+
+    val stateReadDf = spark.read
+      .format("statestore")
+      .option(StateDataSource.PARAM_PATH, checkpointLocation)
+      // explicitly specifying batch ID and operator ID to test out the functionality
+      .option(StateDataSource.PARAM_BATCH_ID, batchId)
+      .option(StateDataSource.PARAM_OPERATOR_ID, operatorId)
+      .load()
+
+    val resultDf = stateReadDf
+      .selectExpr("key.groupKey AS key_groupKey", "value.count AS value_cnt",
+        "value.sum AS value_sum", "value.max AS value_max", "value.min AS value_min")
+
+    checkAnswer(
+      resultDf,
+      Seq(
+        Row(0, 5, 60, 30, 0), // 0, 10, 20, 30
+        Row(1, 5, 65, 31, 1), // 1, 11, 21, 31
+        Row(2, 5, 70, 32, 2), // 2, 12, 22, 32
+        Row(3, 4, 72, 33, 3), // 3, 13, 23, 33
+        Row(4, 4, 76, 34, 4), // 4, 14, 24, 34
+        Row(5, 4, 80, 35, 5), // 5, 15, 25, 35
+        Row(6, 4, 84, 36, 6), // 6, 16, 26, 36
+        Row(7, 4, 88, 37, 7), // 7, 17, 27, 37
+        Row(8, 4, 92, 38, 8), // 8, 18, 28, 38
+        Row(9, 4, 96, 39, 9) // 9, 19, 29, 39
+      )
+    )
+  }
+}
+
+class HDFSBackedStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,
+      classOf[HDFSBackedStateStoreProvider].getName)
+  }
+}
+
+class RocksDBStateDataSourceReadSuite extends StateDataSourceReadSuite {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key,

Review Comment:
   Let's see how much time we will spend if we just duplicate the whole test cases.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1378121683


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
+import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+class StatePartitionReader(
+    storeConf: StateStoreConf,
+    hadoopConf: SerializableConfiguration,
+    partition: StateStoreInputPartition,
+    schema: StructType) extends PartitionReader[InternalRow] {
+
+  private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType]
+  private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType]
+
+  private lazy val store = {
+    val stateStoreId = StateStoreId(partition.stateCheckpointRootLocation,
+      partition.operatorId, partition.partition, partition.storeName)
+    val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
+
+    // TODO: This does not handle the case of session window aggregation; we don't have an

Review Comment:
   I wonder how we handle the case where state metadata doesn't exist(if the query has never run in newer spark version), can we do it this way:
   Check whether state metadata file exists,
   if metadata exists, read the numColsPrefixKey from the metadata
   if metadata doesn't exist, we use 0 and let exception throw if it is session window operator.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1797817269

   cc. @zsxwing @brkyvz @viirya @xuanyuanking Would you mind having a look? Thanks in advance!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43425:
URL: https://github.com/apache/spark/pull/43425#issuecomment-1806616728

   @brkyvz Would you mind having another round of review? Please let me know the above list of follow-up tickets do not address your comments. Thanks!


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "brkyvz (via GitHub)" <gi...@apache.org>.
brkyvz commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1393225661


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")

Review Comment:
   yeah, let's please have a follow up for it



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388799171


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.sql.execution.datasources.v2.state
+
+import java.util
+import java.util.UUID
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.execution.datasources.v2.state.StateDataSource.JoinSideValues.JoinSideValues
+import org.apache.spark.sql.execution.streaming.{CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An implementation of [[TableProvider]] with [[DataSourceRegister]] for State Store data source.
+ */
+class StateDataSource extends TableProvider with DataSourceRegister {
+  import StateDataSource._
+
+  private lazy val session: SparkSession = SparkSession.active
+
+  private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
+
+  override def shortName(): String = "statestore"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+    val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
+    new StateTable(session, schema, sourceOptions, stateConf)
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+    val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+    if (sourceOptions.joinSide != JoinSideValues.none &&
+        sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME) {
+      throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+        s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+    }
+
+    val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
+
+    try {
+      val (keySchema, valueSchema) = sourceOptions.joinSide match {
+        case JoinSideValues.left =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, LeftSide)
+
+        case JoinSideValues.right =>
+          StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
+            sourceOptions.operatorId, RightSide)
+
+        case JoinSideValues.none =>
+          val storeId = new StateStoreId(stateCheckpointLocation.toString, sourceOptions.operatorId,
+            partitionId, sourceOptions.storeName)
+          val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+          val manager = new StateSchemaCompatibilityChecker(providerId, hadoopConf)
+          manager.readSchemaFile()
+      }
+
+      new StructType()
+        .add("key", keySchema)
+        .add("value", valueSchema)
+    } catch {
+      case NonFatal(e) =>
+        throw new IllegalArgumentException("Failed to read the state schema. Either the file " +
+          s"does not exist, or the file is corrupted. options: $sourceOptions", e)
+    }
+  }
+
+  private def buildStateStoreConf(checkpointLocation: String, batchId: Long): StateStoreConf = {
+    val offsetLog = new OffsetSeqLog(session, new Path(checkpointLocation, "offsets").toString)
+    offsetLog.get(batchId) match {
+      case Some(value) =>
+        val metadata = value.metadata.getOrElse(
+          throw new IllegalStateException(s"Metadata is not available for offset log for " +
+            s"$batchId, checkpoint location $checkpointLocation")
+        )
+
+        val clonedRuntimeConf = new RuntimeConfig(session.sessionState.conf.clone())
+        OffsetSeqMetadata.setSessionConf(metadata, clonedRuntimeConf)
+        StateStoreConf(clonedRuntimeConf.sqlConf)
+
+      case _ =>
+        throw new IllegalStateException(s"The offset log for $batchId does not exist, " +
+          s"checkpoint location $checkpointLocation")
+    }
+  }
+
+  override def supportsExternalMetadata(): Boolean = false
+}
+
+object StateDataSource {
+  val PARAM_PATH = "path"
+  val PARAM_BATCH_ID = "batchId"
+  val PARAM_OPERATOR_ID = "operatorId"
+  val PARAM_STORE_NAME = "storeName"
+  val PARAM_JOIN_SIDE = "joinSide"
+
+  object JoinSideValues extends Enumeration {
+    type JoinSideValues = Value
+    val left, right, none = Value
+  }
+
+  case class StateSourceOptions(
+      resolvedCpLocation: String,
+      batchId: Long,
+      operatorId: Int,
+      storeName: String,
+      joinSide: JoinSideValues) {
+    def stateCheckpointLocation: Path = new Path(resolvedCpLocation, "state")
+  }
+
+  object StateSourceOptions {
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        properties: util.Map[String, String]): StateSourceOptions = {
+      apply(sparkSession, hadoopConf, new CaseInsensitiveStringMap(properties))
+    }
+
+    def apply(
+        sparkSession: SparkSession,
+        hadoopConf: Configuration,
+        options: CaseInsensitiveStringMap): StateSourceOptions = {
+      val checkpointLocation = Option(options.get(PARAM_PATH)).orElse {
+        throw new IllegalArgumentException(s"'$PARAM_PATH' must be specified.")
+      }.get
+
+      val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation)
+
+      val batchId = Option(options.get(PARAM_BATCH_ID)).map(_.toLong).orElse {
+        Some(getLastCommittedBatch(sparkSession, resolvedCpLocation))
+      }.get
+
+      if (batchId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_BATCH_ID}' cannot be negative.")
+      }
+
+      val operatorId = Option(options.get(PARAM_OPERATOR_ID)).map(_.toInt)
+        .orElse(Some(0)).get
+
+      if (operatorId < 0) {
+        throw new IllegalArgumentException(s"'${PARAM_OPERATOR_ID}' cannot be negative.")
+      }
+
+      val storeName = Option(options.get(PARAM_STORE_NAME))
+        .getOrElse(StateStoreId.DEFAULT_STORE_NAME)
+
+      if (storeName.isEmpty) {
+        throw new IllegalArgumentException(s"'${PARAM_STORE_NAME}' cannot be an empty string.")
+      }
+
+      val joinSide = try {
+        Option(options.get(PARAM_JOIN_SIDE))
+          .map(JoinSideValues.withName).getOrElse(JoinSideValues.none)
+      } catch {
+        case _: NoSuchElementException =>
+          // convert to IllegalArgumentException
+          throw new IllegalArgumentException(s"Incorrect value of the option " +
+            s"'$PARAM_JOIN_SIDE'. Valid values are ${JoinSideValues.values.mkString(",")}")
+      }
+
+      if (joinSide != JoinSideValues.none && storeName != StateStoreId.DEFAULT_STORE_NAME) {
+        throw new IllegalArgumentException(s"The options '$PARAM_JOIN_SIDE' and " +
+          s"'$PARAM_STORE_NAME' cannot be specified together. Please specify either one.")
+      }
+
+      StateSourceOptions(resolvedCpLocation, batchId, operatorId, storeName, joinSide)
+    }
+
+    private def resolvedCheckpointLocation(
+        hadoopConf: Configuration,
+        checkpointLocation: String): String = {
+      val checkpointPath = new Path(checkpointLocation)
+      val fs = checkpointPath.getFileSystem(hadoopConf)
+      checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString
+    }
+
+    private def getLastCommittedBatch(session: SparkSession, checkpointLocation: String): Long = {
+      val commitLog = new CommitLog(session, new Path(checkpointLocation, "commits").toString)

Review Comment:
   Yeah let's do that. I think this should have been done much earlier, but better late than never.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


Re: [PR] [SPARK-45511][SS] State Data Source - Reader [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43425:
URL: https://github.com/apache/spark/pull/43425#discussion_r1388806314


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala:
##########
@@ -185,6 +187,57 @@ class SymmetricHashJoinStateManager(
     }
   }
 
+  /**
+   * Perform a full scan to provide all available data.
+   *
+   * This produces an iterator over the (key, value, match) tuples. Callers are expected
+   * to consume fully to clean up underlying iterators correctly.

Review Comment:
   Let's file a ticket to address altogether.



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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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