HeartSaVioR commented on a change in pull request #24173: [SPARK-27237][SS]
Introduce State schema validation among query restart
URL: https://github.com/apache/spark/pull/24173#discussion_r267976417
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -477,5 +487,73 @@ object StateStore extends Logging {
None
}
}
-}
+ private class StateSchemaCompatibilityChecker(
+ providerId: StateStoreProviderId,
+ hadoopConf: Configuration) {
+
+ private val storeCpLocation = providerId.storeId.storeCheckpointLocation()
+ private val fm = CheckpointFileManager.create(storeCpLocation, hadoopConf)
+ private val schemaFileLocation = schemaFile(storeCpLocation)
+
+ fm.mkdirs(schemaFileLocation.getParent)
+
+ def check(keySchema: StructType, valueSchema: StructType): Unit = {
+ if (fm.exists(schemaFileLocation)) {
+ logDebug(s"Schema file for provider $providerId exists. Comparing with
provided schema.")
+ val (storedKeySchema, storedValueSchema) = readSchemaFile()
+
+ def typesEq(schema1: StructType, schema2: StructType): Boolean = {
+ val fieldToType: StructField => (DataType, Boolean) = f =>
(f.dataType, f.nullable)
+ (schema1.length == schema2.length) &&
+ schema1.map(fieldToType).equals(schema2.map(fieldToType))
+ }
+
+ val errorMsg = "Provided schema doesn't match to the schema for
existing state! " +
+ "Please note that Spark allow difference of field name: check count
of fields " +
+ "and data type of each field.\n" +
+ s"- provided schema: key $keySchema value $valueSchema\n" +
+ s"- existing schema: key $storedKeySchema value $storedValueSchema\n"
+
+ if (!typesEq(keySchema, storedKeySchema) || !typesEq(valueSchema,
storedValueSchema)) {
+ logError(errorMsg)
+ throw new IllegalStateException(errorMsg)
+ }
+ } else {
+ // schema doesn't exist, create one now
+ logDebug(s"Schema file for provider $providerId doesn't exist.
Creating one.")
+ createSchemaFile(keySchema, valueSchema)
+ }
+ }
+
+ private def readSchemaFile(): (StructType, StructType) = {
+ val inStream = fm.open(schemaFileLocation)
+ val br = new BufferedReader(new InputStreamReader(inStream))
+
+ try {
+ val keySchemaStr = inStream.readUTF().stripSuffix("\n")
+ val valueSchemaStr = inStream.readUTF().stripSuffix("\n")
+
+ (StructType.fromString(keySchemaStr),
StructType.fromString(valueSchemaStr))
+ } finally {
+ br.close()
+ }
+ }
+
+ private def createSchemaFile(keySchema: StructType, valueSchema:
StructType): Unit = {
+ val outStream = fm.createAtomic(schemaFileLocation, overwriteIfPossible
= true)
+ try {
+ outStream.writeUTF(keySchema.json + "\n")
+ outStream.writeUTF(valueSchema.json + "\n")
+ outStream.close()
+ } catch { case NonFatal(e) =>
+ logError(s"Fail to write schema file to $schemaFileLocation", e)
+ outStream.cancel()
+ throw e
+ }
+ }
+ }
+
+ private def schemaFile(storeCpLocation: Path): Path =
+ new Path(new Path(storeCpLocation, "_metadata"), "schema")
Review comment:
So this file will be created and maintained per partition which will avoid
concurrent read/write issue, but may not sound good since it's redundant.
----------------------------------------------------------------
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]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]