You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "chaoqin-li1123 (via GitHub)" <gi...@apache.org> on 2023/11/05 05:18:58 UTC

[PR] [SPARK-45794] [SS] Introduce state metadata source to query the streaming state metadata information [spark]

chaoqin-li1123 opened a new pull request, #43660:
URL: https://github.com/apache/spark/pull/43660

   ### What changes were proposed in this pull request?
   Introduce a new data source so that user can query the metadata of each state store of a streaming query, the schema of the result will be
   operatorId INT | operatorName STRING | stateStoreName STRING | numPartitions INT | numColsPrefixKey INT | minBatchId  LONG | minBatchId LONG
   To use this source, specify the source format and checkpoint path and load the dataframe
   df = spark.read.format(“state-metadata”).load(“/checkpointPath”)
   
   ### Why are the changes needed?
   To improve debugability. Also facilitate the query of state store data source introduced in SPARK-45511 by displaying the operator id, batch id and state store name.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, this is a new source exposed to user.
   
   ### How was this patch tested?
   Add test to verify the output of state metadata
   
   ### 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] [SPARK-45794] [SS] Introduce state metadata source to query the streaming state metadata information [spark]

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

   @HeartSaVioR PTAL, 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-45794] [SS] Introduce state metadata source to query the streaming state metadata information [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataV1}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.SerializableConfiguration
+
+case class StateMetadataTableEntry(
+    operatorId: Long,
+    operatorName: String,
+    stateStoreName: String,
+    numPartitions: Int,
+    numColsPrefixKey: Int,
+    minBatchId: Long,
+    maxBatchId: Long) {
+  def toRow(): InternalRow = {
+    InternalRow.fromSeq(
+      Seq(operatorId,
+        UTF8String.fromString(operatorName),
+        UTF8String.fromString(stateStoreName),
+        numPartitions,
+        numColsPrefixKey,
+        minBatchId,
+        maxBatchId))
+  }
+}
+
+object StateMetadataTableEntry {
+  private[sql] val schema = {
+    new StructType()
+      .add("operatorId", LongType)
+      .add("operatorName", StringType)
+      .add("stateStoreName", StringType)
+      .add("numPartitions", IntegerType)
+      .add("numColsPrefixKey", IntegerType)
+      .add("minBatchId", LongType)
+      .add("maxBatchId", LongType)
+  }
+}
+
+class StateMetadataSource extends TableProvider with DataSourceRegister {
+  override def shortName(): String = "state-metadata"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    new StateMetadataTable
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    // The schema of state metadata table is static.
+   StateMetadataTableEntry.schema
+  }
+}
+
+
+class StateMetadataTable extends Table with SupportsRead {
+  override def name(): String = "state-metadata-table"
+
+  override def schema(): StructType = StateMetadataTableEntry.schema
+
+  override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+    () => {
+      assert(options.containsKey("path"), "Must specify checkpoint path to read state metadata")

Review Comment:
   We should throw an `IllegalArgumentException` or proper error class. Let's do former and we can apply error class altogether for state data source & state metadata data source.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataV1}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.SerializableConfiguration
+
+case class StateMetadataTableEntry(
+    operatorId: Long,
+    operatorName: String,
+    stateStoreName: String,
+    numPartitions: Int,
+    numColsPrefixKey: Int,
+    minBatchId: Long,
+    maxBatchId: Long) {
+  def toRow(): InternalRow = {
+    InternalRow.fromSeq(
+      Seq(operatorId,
+        UTF8String.fromString(operatorName),
+        UTF8String.fromString(stateStoreName),
+        numPartitions,
+        numColsPrefixKey,
+        minBatchId,
+        maxBatchId))
+  }
+}
+
+object StateMetadataTableEntry {
+  private[sql] val schema = {
+    new StructType()
+      .add("operatorId", LongType)
+      .add("operatorName", StringType)
+      .add("stateStoreName", StringType)
+      .add("numPartitions", IntegerType)
+      .add("numColsPrefixKey", IntegerType)

Review Comment:
   Can we make this be a metadata column? Probably adding a underbar as prefix as well - `_numColsPrefixKey`.
   
   This is purely an internal one and most users won't have a context for this. We never require users to know 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] [SPARK-45794] [SS] Introduce state metadata source to query the streaming state metadata information [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataV1}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.SerializableConfiguration
+
+case class StateMetadataTableEntry(
+    operatorId: Long,
+    operatorName: String,
+    stateStoreName: String,
+    numPartitions: Int,
+    numColsPrefixKey: Int,
+    minBatchId: Long,
+    maxBatchId: Long) {
+  def toRow(): InternalRow = {
+    InternalRow.fromSeq(
+      Seq(operatorId,
+        UTF8String.fromString(operatorName),
+        UTF8String.fromString(stateStoreName),
+        numPartitions,
+        numColsPrefixKey,
+        minBatchId,
+        maxBatchId))
+  }
+}
+
+object StateMetadataTableEntry {
+  private[sql] val schema = {
+    new StructType()
+      .add("operatorId", LongType)
+      .add("operatorName", StringType)
+      .add("stateStoreName", StringType)
+      .add("numPartitions", IntegerType)
+      .add("numColsPrefixKey", IntegerType)
+      .add("minBatchId", LongType)
+      .add("maxBatchId", LongType)
+  }
+}
+
+class StateMetadataSource extends TableProvider with DataSourceRegister {
+  override def shortName(): String = "state-metadata"
+
+  override def getTable(
+      schema: StructType,
+      partitioning: Array[Transform],
+      properties: util.Map[String, String]): Table = {
+    new StateMetadataTable
+  }
+
+  override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
+    // The schema of state metadata table is static.
+   StateMetadataTableEntry.schema
+  }
+}
+
+
+class StateMetadataTable extends Table with SupportsRead {
+  override def name(): String = "state-metadata-table"
+
+  override def schema(): StructType = StateMetadataTableEntry.schema
+
+  override def capabilities(): util.Set[TableCapability] = Set(TableCapability.BATCH_READ).asJava
+
+  override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+    () => {
+      assert(options.containsKey("path"), "Must specify checkpoint path to read state metadata")

Review Comment:
   Fixed.



-- 
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-45794][SS] Introduce state metadata source to query the streaming state metadata information [spark]

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

   Thanks! 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-45794] [SS] Introduce state metadata source to query the streaming state metadata information [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, PathFilter}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider}
+import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
+import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataV1}
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.SerializableConfiguration
+
+case class StateMetadataTableEntry(
+    operatorId: Long,
+    operatorName: String,
+    stateStoreName: String,
+    numPartitions: Int,
+    numColsPrefixKey: Int,
+    minBatchId: Long,
+    maxBatchId: Long) {
+  def toRow(): InternalRow = {
+    InternalRow.fromSeq(
+      Seq(operatorId,
+        UTF8String.fromString(operatorName),
+        UTF8String.fromString(stateStoreName),
+        numPartitions,
+        numColsPrefixKey,
+        minBatchId,
+        maxBatchId))
+  }
+}
+
+object StateMetadataTableEntry {
+  private[sql] val schema = {
+    new StructType()
+      .add("operatorId", LongType)
+      .add("operatorName", StringType)
+      .add("stateStoreName", StringType)
+      .add("numPartitions", IntegerType)
+      .add("numColsPrefixKey", IntegerType)

Review Comment:
   Makes sense, I make it a 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-45794][SS] Introduce state metadata source to query the streaming state metadata information [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #43660: [SPARK-45794][SS] Introduce state metadata source to query the streaming state metadata information
URL: https://github.com/apache/spark/pull/43660


-- 
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-45794][SS] Introduce state metadata source to query the streaming state metadata information [spark]

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

   CI only failed from [Run / Build modules: pyspark-mllib, pyspark-ml, pyspark-ml-connect](https://github.com/chaoqin-li1123/spark/actions/runs/6790713907/job/18461592589#logs) which is irrelevant.


-- 
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-45794][SS] Introduce state metadata source to query the streaming state metadata information [spark]

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

   Filed a JIRA ticket https://issues.apache.org/jira/browse/SPARK-45833 for addressing documentation on both state data source and state metadata source.


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