You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/03/21 22:18:53 UTC

[GitHub] [spark] HeartSaVioR commented on a change in pull request #24173: [SPARK-27237][SS] Introduce State schema validation among query restart

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_r267975963
 
 

 ##########
 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 {
 
 Review comment:
   Maybe we want to overwrite schema if only field name is changed. Even we don't leverage field name to check compatibility, storing and showing the name would give more meaningful message to end users.
   
   My 2 cents, we might also want to log (with proper level) when only field name is changed - there's a chance end users intend to change field's name, but there's also some chance for state to be semantically broken when fields with same data type are swapped, etc. But this is pretty optional and up to our preference.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org