viirya commented on a change in pull request #24173:
URL: https://github.com/apache/spark/pull/24173#discussion_r530163299



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1294,6 +1294,13 @@ object SQLConf {
       .createWithDefault(
         
"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider")
 
+  val STATE_SCHEMA_CHECK_ENABLED =
+    buildConf("spark.sql.streaming.stateStore.stateSchemaCheck")
+      .doc("When true, Spark will validate the state schema against schema on 
existing state and " +
+        "fail query if it's incompatible.")
+      .booleanConf

Review comment:
       .version("3.1.0")?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala
##########
@@ -150,6 +172,25 @@ private class StateStoreCoordinator(override val rpcEnv: 
RpcEnv)
         storeIdsToRemove.mkString(", "))
       context.reply(true)
 
+    case ValidateSchema(providerId, keySchema, valueSchema, checkEnabled) =>
+      // normalize partition ID to validate only once for one state operator
+      val newProviderId = 
StateStoreProviderId.withNoPartitionInformation(providerId)
+
+      val result = schemaValidated.getOrElseUpdate(newProviderId, {
+        val checker = new StateSchemaCompatibilityChecker(newProviderId, 
hadoopConf)
+
+        // regardless of configuration, we check compatibility to at least 
write schema file
+        // if necessary
+        val ret = Try(checker.check(keySchema, 
valueSchema)).toEither.fold(Some(_), _ => None)

Review comment:
       Roughly remember we don't recommend using `Try` in Spark?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala
##########
@@ -150,6 +172,25 @@ private class StateStoreCoordinator(override val rpcEnv: 
RpcEnv)
         storeIdsToRemove.mkString(", "))
       context.reply(true)
 
+    case ValidateSchema(providerId, keySchema, valueSchema, checkEnabled) =>
+      // normalize partition ID to validate only once for one state operator
+      val newProviderId = 
StateStoreProviderId.withNoPartitionInformation(providerId)
+

Review comment:
       I think you already do `withNoPartitionInformation` for the `providerId` 
before you call `validateSchema`?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
##########
@@ -0,0 +1,120 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{StructField, StructType}
+
+case class StateSchemaNotCompatible(message: String) extends Exception(message)
+
+class StateSchemaCompatibilityChecker(
+    providerId: StateStoreProviderId,
+    hadoopConf: Configuration) extends Logging {
+
+  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 fieldCompatible(fieldOld: StructField, fieldNew: StructField): 
Boolean = {
+        // compatibility for nullable
+        // - same: OK
+        // - non-nullable -> nullable: OK
+        // - nullable -> non-nullable: Not compatible
+        (fieldOld.dataType == fieldNew.dataType) &&
+          ((fieldOld.nullable == fieldNew.nullable) ||
+            (!fieldOld.nullable && fieldNew.nullable))
+      }
+
+      def schemaCompatible(schemaOld: StructType, schemaNew: StructType): 
Boolean = {
+        (schemaOld.length == schemaNew.length) &&
+          schemaOld.zip(schemaNew).forall { case (f1, f2) => 
fieldCompatible(f1, f2) }
+      }

Review comment:
       Hmm, does `schemaCompatible` work for nested fields? Looks like you only 
compare top-level fields with `fieldCompatible`.




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