HeartSaVioR commented on a change in pull request #32788:
URL: https://github.com/apache/spark/pull/32788#discussion_r647093866



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
##########
@@ -34,6 +35,7 @@ class StateSchemaCompatibilityChecker(
   private val storeCpLocation = providerId.storeId.storeCheckpointLocation()
   private val fm = CheckpointFileManager.create(storeCpLocation, hadoopConf)
   private val schemaFileLocation = schemaFile(storeCpLocation)
+  private val schemaWriter = SchemaWriter.createSchemaWriter("v2")

Review comment:
       Let's make this simple to pass StateSchemaCompatibilityChecker.VERSION 
instead. Doesn't need to let this class know how the version string looks like.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.streaming.state
+
+import java.io.StringReader
+
+import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream}
+
+import org.apache.spark.sql.execution.streaming.MetadataVersionUtil
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+
+/**
+ * Helper classes for reading/writing state schema.
+ */
+object SchemaHelper {
+
+  sealed trait SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType)
+  }
+
+  object SchemaReader {
+    def createSchemaReader(schemaVersion: String): SchemaReader = {
+      val version = MetadataVersionUtil.validateVersion(schemaVersion,
+        StateSchemaCompatibilityChecker.VERSION)
+      version match {
+        case 1 => new SchemaV1Reader
+        case 2 => new SchemaV2Reader
+      }
+    }
+  }
+
+  class SchemaV1Reader extends SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType) = {
+      val keySchemaStr = inputStream.readUTF()
+      val valueSchemaStr = inputStream.readUTF()
+      (StructType.fromString(keySchemaStr), 
StructType.fromString(valueSchemaStr))
+    }
+  }
+
+  class SchemaV2Reader extends SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType) = {
+      val buf = new StringBuilder
+      val numKeyChunks = inputStream.readInt()
+      (0 until numKeyChunks).foreach(_ => buf.append(inputStream.readUTF()))
+      val keySchemaStr = buf.toString()
+
+      buf.clear()
+      val numValueChunks = inputStream.readInt()
+      (0 until numValueChunks).foreach(_ => buf.append(inputStream.readUTF()))
+      val valueSchemaStr = buf.toString()
+      (StructType.fromString(keySchemaStr), 
StructType.fromString(valueSchemaStr))
+    }
+  }
+
+  trait SchemaWriter {
+    val version: String

Review comment:
       The representation of version won't be changed, so "v" can just be 
directly referred in `writeVersion` and this can be `Int`.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.streaming.state
+
+import java.io.StringReader
+
+import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream}
+
+import org.apache.spark.sql.execution.streaming.MetadataVersionUtil
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+
+/**
+ * Helper classes for reading/writing state schema.
+ */
+object SchemaHelper {
+
+  sealed trait SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType)
+  }
+
+  object SchemaReader {
+    def createSchemaReader(schemaVersion: String): SchemaReader = {
+      val version = MetadataVersionUtil.validateVersion(schemaVersion,
+        StateSchemaCompatibilityChecker.VERSION)
+      version match {
+        case 1 => new SchemaV1Reader
+        case 2 => new SchemaV2Reader
+      }
+    }
+  }
+
+  class SchemaV1Reader extends SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType) = {
+      val keySchemaStr = inputStream.readUTF()
+      val valueSchemaStr = inputStream.readUTF()
+      (StructType.fromString(keySchemaStr), 
StructType.fromString(valueSchemaStr))
+    }
+  }
+
+  class SchemaV2Reader extends SchemaReader {
+    def read(inputStream: FSDataInputStream): (StructType, StructType) = {
+      val buf = new StringBuilder
+      val numKeyChunks = inputStream.readInt()
+      (0 until numKeyChunks).foreach(_ => buf.append(inputStream.readUTF()))
+      val keySchemaStr = buf.toString()
+
+      buf.clear()
+      val numValueChunks = inputStream.readInt()
+      (0 until numValueChunks).foreach(_ => buf.append(inputStream.readUTF()))
+      val valueSchemaStr = buf.toString()
+      (StructType.fromString(keySchemaStr), 
StructType.fromString(valueSchemaStr))
+    }
+  }
+
+  trait SchemaWriter {
+    val version: String
+
+    final def write(
+        keySchema: StructType,
+        valueSchema: StructType,
+        outputStream: FSDataOutputStream): Unit = {
+      writeVersion(outputStream)
+      writeSchema(keySchema, valueSchema, outputStream)
+    }
+
+    private def writeVersion(outputStream: FSDataOutputStream): Unit = {
+      outputStream.writeUTF(version)
+    }
+
+    protected def writeSchema(
+        keySchema: StructType,
+        valueSchema: StructType,
+        outputStream: FSDataOutputStream): Unit
+  }
+
+  object SchemaWriter {
+    def createSchemaWriter(schemaVersion: String): SchemaWriter = {
+      val version = MetadataVersionUtil.validateVersion(schemaVersion,

Review comment:
       I don't think this is necessary for writer path. You can always pass the 
version "number".




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



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

Reply via email to