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 2020/06/02 18:35:32 UTC

[GitHub] [spark] xuanyuanking opened a new pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

xuanyuanking opened a new pull request #28707:
URL: https://github.com/apache/spark/pull/28707


   ### What changes were proposed in this pull request?
   Introduce UnsafeRow format validation for streaming state store.
   
   ### Why are the changes needed?
   Currently, Structured Streaming directly puts the UnsafeRow into StateStore without any schema validation. It's a dangerous behavior when users reusing the checkpoint file during migration. Any changes or bug fix related to the aggregate function may cause random exceptions, even the wrong answer, e.g SPARK-28067.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. If the underlying changes are detected when the checkpoint is reused during migration, the InvalidUnsafeRowException will be thrown.
   
   
   ### How was this patch tested?
   UT added. Will also add integrated tests for more scenario in another PR separately.
   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638180171


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123475/
   Test FAILed.


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434226399



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -94,6 +110,28 @@ abstract class StreamingAggregationStateManagerBaseImpl(
     // discard and don't convert values to avoid computation
     store.getRange(None, None).map(_.key)
   }
+
+  override def unsafeRowFormatValidation(row: UnsafeRow, schema: StructType): Unit = {
+    if (checkFormat && SQLConf.get.getConf(
+        SQLConf.STREAMING_STATE_FORMAT_CHECK_ENABLED) && row != null) {
+      if (schema.fields.length != row.numFields) {

Review comment:
       This method exposes implementation details of UnsafeRow directly. Could we please let UnsafeRow have such check method? UnsafeRow itself is aware of data types so the check method can receive the list of data types and do the assertion by its own.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_STATE_FORMAT_CHECK_ENABLED =

Review comment:
       This is misleading - we're only detecting the case from streaming aggregation.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -77,13 +82,24 @@ object StreamingAggregationStateManager extends Logging {
   }
 }
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The UnsafeRow format is invalid. This may happen when using the old " +
+    "version or broken checkpoint file. To resolve this problem, you can try to restart the " +

Review comment:
       I'm not sure I understand with the possible root causes and the proposed solutions. The problem comes either schema is incompatible (probably due to the change of the query, or change of the underlying aggregation function) or row is corrupted, which any solution described here can not resolve.
   
   "Old version" here is ambiguous, because there's another semantic of "version" here, state format, which is not expected to introduce such incompatible format issue. Did you see the case?




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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639239790


   My alternative with wrapping state store is something like below:
   
   ```
     class RowValidatingStateStore(
         underlying: StateStore,
         keyType: Seq[DataType],
         valueType: Seq[DataType]) extends StateStore {
       private var isValidated = false
   
       override def get(key: UnsafeRow): UnsafeRow = {
         val value = underlying.get(key)
         if (!isValidated) {
           validateRow(value, valueType)
           isValidated = true
         }
         value
       }
   
       override def id: StateStoreId = underlying.id
       override def version: Long = underlying.version
       override def put(key: UnsafeRow, value: UnsafeRow): Unit = underlying.put(key, value)
       override def remove(key: UnsafeRow): Unit = underlying.remove(key)
       override def commit(): Long = underlying.commit()
       override def abort(): Unit = underlying.abort()
       override def iterator(): Iterator[UnsafeRowPair] = underlying.iterator()
       override def metrics: StateStoreMetrics = underlying.metrics
       override def hasCommitted: Boolean = underlying.hasCommitted
   
       private def validateRow(row: UnsafeRow, rowDataType: Seq[DataType]): Unit = {
         // TODO: call util method with row and data type to validate - note that it can only check with value schema
       }
     }
   
     def get(...): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         ...
       }
       // TODO: add if statement to see whether it should wrap state store or not
       new RowValidatingStateStore(storeProvider.getStore(version, keySchema, valueSchema))
     }
   ```
   
   The example code only checks in get operation, which is insufficient to check "key" row in state. That said, iterator approach still provides more possibility of validation, though the validation of unsafe row itself doesn't have enough coverage of checking various incompatibility issues (Definitely we should have another guards as well) so that's a sort of OK to only cover value side.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638644781


   > What are other stateful operations that use unsafe row? I think we can apply the check everywhere.
   
   State store itself stores UnsafeRow, hence it applies to everywhere in stateful operations. I'd propose to do it like https://github.com/apache/spark/pull/28707#issuecomment-637926400 instead of fixing everywhere.


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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637991937


   @skambha  it doesn't fix the issue, it gives a better error message when we hit the issue.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441492670



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
##########
@@ -259,6 +259,7 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
   @volatile private var storeConf: StateStoreConf = _
   @volatile private var hadoopConf: Configuration = _
   @volatile private var numberOfVersionsToRetainInMemory: Int = _
+  @volatile private var isValidated = false

Review comment:
       Can we add a TODO that this validation should be moved to a higher level so that it works from all state store implementations?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
##########
@@ -259,6 +259,7 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
   @volatile private var storeConf: StateStoreConf = _
   @volatile private var hadoopConf: Configuration = _
   @volatile private var numberOfVersionsToRetainInMemory: Int = _
+  @volatile private var isValidated = false

Review comment:
       Can we add a TODO that this validation should be moved to a higher level so that it works for all state store implementations?




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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639200645


   > @HeartSaVioR After taking a further look. Instead of dealing with the iterator, how about adding the invalidation for all state store operations in StateStoreProvider? Since we can get the key/value row during load map. WDYT?
   
   It would be nice to see the proposed change by code to avoid misunderstanding, like I proposed in previous comment. (anything including commit in your fork or text comment is OK) I'll try out my alternative (wrapping State Store) and show the code change. 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.

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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441919301



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1237,6 +1237,15 @@ object SQLConf {
       .intConf
       .createWithDefault(10)
 
+  val STATE_STORE_FORMAT_VALIDATION_ENABLED =
+    buildConf("spark.sql.streaming.stateStore.formatValidation.enabled")
+      .internal()
+      .doc("When true, check if the UnsafeRow from the state store is valid or not when running " +
+        "streaming queries. This can happen if the state store format has been changed.")

Review comment:
       Make senes, done in 557eb30.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645306547


   **[Test build #124166 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124166/testReport)** for PR 28707 at commit [`01007fb`](https://github.com/apache/spark/commit/01007fb9f03c003bfc00d2e2358c9029b83f16e6).


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645715888


   **[Test build #124186 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124186/testReport)** for PR 28707 at commit [`557eb30`](https://github.com/apache/spark/commit/557eb3099b3d0abe1fd2d7d91754fa747e05d200).


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441919193



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+
+object UnsafeRowUtils {
+
+  /**
+   * Use the following rules to check the integrity of the UnsafeRow:
+   * - schema.fields.length == row.numFields should always be true
+   * - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < row.getSizeInBytes should always be
+   *   true if the expectedSchema contains at least one field.
+   * - For variable-length fields: if null bit says it's null then don't do anything, else extract
+   *   offset and size:
+   *   1) 0 <= size < row.getSizeInBytes should always be true. We can be even more precise than
+   *      this, where the upper bound of size can only be as big as the variable length part of
+   *      the row.
+   *   2) offset should be >= fixed sized part of the row.
+   *   3) offset + size should be within the row bounds.
+   * - For fixed-length fields that are narrower than 8 bytes (boolean/byte/short/int/float), if
+   *   null bit says it's null then don't do anything, else:
+   *     check if the unused bits in the field are all zeros. The UnsafeRowWriter's write() methods
+   *     make this guarantee.
+   * - Check the total length of the row.
+   */
+  def validateStructuralIntegrity(row: UnsafeRow, expectedSchema: StructType): Boolean = {
+    if (expectedSchema.fields.length != row.numFields) {
+      return false
+    }
+    val bitSetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields)
+    val rowSizeInBytes = row.getSizeInBytes
+    if (expectedSchema.fields.length > 0 && bitSetWidthInBytes >= rowSizeInBytes) {
+      return false
+    }
+    var varLenFieldsSizeInBytes = 0
+    expectedSchema.fields.zipWithIndex.foreach {
+      case (field, index) if !UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        val offsetAndSize = row.getLong(index)
+        val offset = (offsetAndSize >> 32).toInt
+        val size = offsetAndSize.toInt
+        if (size < 0 ||
+            offset < bitSetWidthInBytes + 8 * row.numFields || offset + size > rowSizeInBytes) {
+          return false
+        }
+        varLenFieldsSizeInBytes += size
+      case (field, index) if UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        field.dataType match {
+          case BooleanType =>
+            if ((row.getLong(index) >> 1) != 0L) return false
+          case ByteType =>
+            if ((row.getLong(index) >> 8) != 0L) return false
+          case ShortType =>
+            if ((row.getLong(index) >> 16) != 0L) return false
+          case IntegerType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case FloatType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case _ =>
+        }
+      case (field, index) if field.dataType == NullType =>

Review comment:
       Thanks for the explanation, done in 557eb30.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645433621


   **[Test build #124171 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124171/testReport)** for PR 28707 at commit [`fd74ff9`](https://github.com/apache/spark/commit/fd74ff9c337d06f4cb4ccfc638d837b5ea3d0e11).


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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638679010


   @skambha You can check the integrated tests in #28725. If we delete the validation, we'll get a NPE for [this test](https://github.com/apache/spark/pull/28725/files#diff-492f0d70824a58ef2ea94a54dc6f9707R79), and get an assertion in the unsafe row for [this test](https://github.com/apache/spark/pull/28725/files#diff-492f0d70824a58ef2ea94a54dc6f9707R185). That is to say, we will get random failures during reusing the checkpoint written by the old Spark version.


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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-643916110


   cc @maropu @gatorsmile @HeartSaVioR @dongjoon-hyun 
   
   A new regression bug SPARK-31990 was found when investigating the test failure https://github.com/apache/spark/pull/28707#issuecomment-639861273. The root cause is that [this line](https://github.com/apache/spark/pull/28062/files#diff-7a46f10c3cedbf013cf255564d9483cdL2458) in SPARK-31292 made the order of groupCols in Deduplicate changed, and the order changing will break the validation logic here. That is to say, if we don't have this PR, the executor JVM could probably crash, throw a random exception, or even return a wrong answer when using the checkpoint written by the previous version.
   
   So we have 2 related work of this PR:
   
   - [ ]**[Block]** Fix and merge the compatibility issue in #28830
   - [ ][Follow-up] Add new test(or modify the current Kafka test) in #28725
   
   ------------------
   ### More detailed analysis:
   The expected order of `Deduplicate.groupCols` in UT KafkaMicroBatchV2SourceSuite is
   ```
   [timestamp, partition, timestampType, key, offset, topic, value]
   ```
   Which is also the order in the checkpoint written by the version before Spark 3.0
   After the changes in SPARK-31292, the groupCols changed to
   ```
   [key, value, topic, partition, offset, timestamp, timestampType]
   ```
   
   #### Why this incompatibility bug didn't fail the `KafkaMicroBatchV2SourceSuite` when it merged?
   
   Because the UT `default config of includeHeader doesn't break the existing query from Spark 2.4` didn't test the scenario of duplicating and check the answer.
   Although the UT uses the checkpoint written by version 2.4.3 and streaming duplicate operation, it just wants to prove that the new header(added in SPARK-23539) doesn't break the original checkpoint file. 


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637926400


   And personally I'd rather do the check in StateStore with additional overhead of reading "a" row in prior to achieve the same in all stateful operations. 
   
   ```
     /** Get or create a store associated with the id. */
     def get(
         storeProviderId: StateStoreProviderId,
         keySchema: StructType,
         valueSchema: StructType,
         indexOrdinal: Option[Int],
         version: Long,
         storeConf: StateStoreConf,
         hadoopConf: Configuration): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         startMaintenanceIfNeeded()
         val provider = loadedProviders.getOrElseUpdate(
           storeProviderId,
           StateStoreProvider.createAndInit(
             storeProviderId.storeId, keySchema, valueSchema, indexOrdinal, storeConf, hadoopConf)
         )
         reportActiveStoreInstance(storeProviderId)
         provider
       }
       val store = storeProvider.getStore(version)
       val iter = store.iterator()
       if (iter.nonEmpty) {
         val rowPair = iter.next()
         val key = rowPair.key
         val value = rowPair.value
         // TODO: validate key with key schema
         // TODO: validate value with value schema
       }
       store
     }
   ```
   
   For streaming aggregations it initializes "two" state stores so the overhead goes to "two" rows, but I don't think the overhead matters much.
   
   If we really concern about the overhead of making additional "iterator", just have a StateStore wrapper wrapping `store` and do the same - only validate once for the first "get". In either way, we never need to restrict the functionality to the streaming aggregation.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638065366






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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638072075


   > @skambha it doesn't fix the issue, it gives a better error message when we hit the issue.
   
   Yep, WIP for the integrated test of the state store format invalidation. I will show you the difference with/ this patch on the error message.
   
   > Safety guards must be placed in front of this - like SPARK-27237, which I think it covers various general issues with providing clearer guide of schema incompatibility between state and the query being run.
   
   Yes, the two approach addresses different sides of this issue, SPARK-27237 require an extra file to keep the schema, which can make the schema checking possible. This one is a guard for random failure or correctness bug.


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



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


[GitHub] [spark] cloud-fan edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637992953


   I think this PR and SPARK-27237 are orthogonal, and we should have both. SPARK-27237 is a bit hard to be merged as it changes the checkpoint. We may need more reviews to see if it's future proof (e.g. when we want to support schema evolution of the state store).
   
   Anyway, this PR covers the cases that people upgrade from Spark 2.x to 3.x, which is necessary even if we have SPARK-27237.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645715888


   **[Test build #124186 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124186/testReport)** for PR 28707 at commit [`557eb30`](https://github.com/apache/spark/commit/557eb3099b3d0abe1fd2d7d91754fa747e05d200).


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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638072075


   > @skambha it doesn't fix the issue, it gives a better error message when we hit the issue.
   
   Yep, I WIP for the integrated test of the state store format invalidation. I will show you the difference with/ this patch on the error message.
   
   > Safety guards must be placed in front of this - like SPARK-27237, which I think it covers various general issues with providing clearer guide of schema incompatibility between state and the query being run.
   
   Yes, the two approach addresses different sides of this issue, SPARK-27237 require an extra file to keep the schema, which can make the schema checking possible. This one is a guard for random failure or correctness bug.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434423252



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_AGGREGATION_STATE_FORMAT_CHECK_ENABLED =
+    buildConf("spark.sql.streaming.aggregationStateFormatCheck.enabled")
+      .doc("Whether to detect a streaming aggregation query may try to use an invalid UnsafeRow " +

Review comment:
       nit: `When true, check if the UnsafeRow from the state store is valid or not when running streaming aggregation queries. This can happen if the state store format has been changed.`




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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441615184



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+
+object UnsafeRowUtils {
+
+  /**
+   * Use the following rules to check the integrity of the UnsafeRow:
+   * - schema.fields.length == row.numFields should always be true
+   * - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < row.getSizeInBytes should always be
+   *   true if the expectedSchema contains at least one field.
+   * - For variable-length fields: if null bit says it's null then don't do anything, else extract
+   *   offset and size:
+   *   1) 0 <= size < row.getSizeInBytes should always be true. We can be even more precise than
+   *      this, where the upper bound of size can only be as big as the variable length part of
+   *      the row.
+   *   2) offset should be >= fixed sized part of the row.
+   *   3) offset + size should be within the row bounds.
+   * - For fixed-length fields that are narrower than 8 bytes (boolean/byte/short/int/float), if
+   *   null bit says it's null then don't do anything, else:
+   *     check if the unused bits in the field are all zeros. The UnsafeRowWriter's write() methods
+   *     make this guarantee.
+   * - Check the total length of the row.
+   */
+  def validateStructuralIntegrity(row: UnsafeRow, expectedSchema: StructType): Boolean = {
+    if (expectedSchema.fields.length != row.numFields) {
+      return false
+    }
+    val bitSetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields)
+    val rowSizeInBytes = row.getSizeInBytes
+    if (expectedSchema.fields.length > 0 && bitSetWidthInBytes >= rowSizeInBytes) {
+      return false
+    }
+    var varLenFieldsSizeInBytes = 0
+    expectedSchema.fields.zipWithIndex.foreach {
+      case (field, index) if !UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        val offsetAndSize = row.getLong(index)
+        val offset = (offsetAndSize >> 32).toInt
+        val size = offsetAndSize.toInt
+        if (size < 0 ||
+            offset < bitSetWidthInBytes + 8 * row.numFields || offset + size > rowSizeInBytes) {
+          return false
+        }
+        varLenFieldsSizeInBytes += size
+      case (field, index) if UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        field.dataType match {
+          case BooleanType =>
+            if ((row.getLong(index) >> 1) != 0L) return false
+          case ByteType =>
+            if ((row.getLong(index) >> 8) != 0L) return false
+          case ShortType =>
+            if ((row.getLong(index) >> 16) != 0L) return false
+          case IntegerType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case FloatType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case _ =>
+        }
+      case _ =>

Review comment:
       Thanks, done in fd74ff9.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638065366






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645804908






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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637735027


   **[Test build #123447 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123447/testReport)** for PR 28707 at commit [`4d14961`](https://github.com/apache/spark/commit/4d14961499a23498c8af0fabd842a1ed46e6cfc7).


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638520229


   Will this be included to Spark 3.0.0? If this is to unblock SPARK-28067 to be included to Spark 3.0.0 then it's OK to consider this first, but if this plans to go to Spark 3.1 then I'm not sure about the priority - are all of you aware that the PR for SPARK-27237 was submitted more than a year ago, and still be considered as later?
   
   I still don't get why the proposal is restricting its usage to streaming aggregation, whereas the mechanism is a validation of the UnsafeRow which can be applied to all stateful operations. Let's not to pinpoint the problem we've just seen.
   
   Also from my side the overhead of the validation logic looks to be trivial compared to the operations stateful operators will take - we don't do the validation for all rows, even don't sample, just the first one. Unless we have a chance to bring a show-stopper bug in the validation logic (so that we need to provide the way to disable the validation), I'm not seeing the needs of new configuration.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637735695






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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638076637


   **[Test build #123477 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123477/testReport)** for PR 28707 at commit [`10a7980`](https://github.com/apache/spark/commit/10a7980b332c57e77f022fced1cba8768c8e4ff6).


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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644635176


   I guess we may need to change the state store API to allow Spark to do validation easily. I'm OK to put the validation logic in the state store implementation for now, and figure out how to change the state store API later.


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639873447


   Sorry my comment was edited so you may be missed the content, but it is also a sort of pointing out of "pinpoint" - do you think your approach works with other state store providers as well? The root cause isn't bound to the implementation of state store provider but this patch is only addressing HDFS state store provider.
   
   I guess you're trying to find how it can be done less frequently, first time the state is loaded from the file, which is optimal. While I think it can be even done without binding to the state store provider implementation if we really need it, have we measured the actual overhead? If the overhead turns out to be trivial then it won't be matter we run validation check for each batch. It sounds to be sub-optimal, but the overhead would be trivial.


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441617572



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -143,6 +145,16 @@ case class StateStoreCustomSumMetric(name: String, desc: String) extends StateSt
 case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
 case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected in state store.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The streaming query failed by state format invalidation. " +
+    "The following reasons may cause this: 1. An old Spark version wrote the checkpoint that is " +
+    "incompatible with the current one; 2. Broken checkpoint files; 3. The query is changed " +
+    "among restart. For the first case, you can try to restart the application without " +

Review comment:
       The resolution is for the first case. For the rest cases listing, they should be considered as user problems.




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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645804113


   **[Test build #124186 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124186/testReport)** for PR 28707 at commit [`557eb30`](https://github.com/apache/spark/commit/557eb3099b3d0abe1fd2d7d91754fa747e05d200).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639811979


   All the comments addressed in 1f71563. Thanks for the review!
   It also includes my alternative of adding the invalidation for all state store operations in StateStoreProvider, PTAL.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639812505


   **[Test build #123580 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123580/testReport)** for PR 28707 at commit [`7a5e09a`](https://github.com/apache/spark/commit/7a5e09a3d52cc6fa0c5aad0aa1e3c84878afe656).


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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-643916110


   cc @maropu @gatorsmile @HeartSaVioR @dongjoon-hyun 
   
   A new regression bug SPARK-31990 was found when investigating the test failure https://github.com/apache/spark/pull/28707#issuecomment-639861273. The root cause is that [this line](https://github.com/apache/spark/pull/28062/files#diff-7a46f10c3cedbf013cf255564d9483cdL2458) in SPARK-31292 made the order of groupCols in Deduplicate changed, and the order changing will break the validation logic here. That is to say, if we don't have this PR, the executor JVM could probably crash, throw a random exception, or even return a wrong answer when using the checkpoint written by the previous version.
   
   So we have 2 related work of this PR:
   
   - [ ] **[Block]** Fix and merge the compatibility issue in #28830
   - [ ] [Follow-up] Add new test(or modify the current Kafka test) in #28725
   
   ------------------
   ### More detailed analysis:
   The expected order of `Deduplicate.groupCols` in UT KafkaMicroBatchV2SourceSuite is
   ```
   [timestamp, partition, timestampType, key, offset, topic, value]
   ```
   Which is also the order in the checkpoint written by the version before Spark 3.0
   After the changes in SPARK-31292, the groupCols changed to
   ```
   [key, value, topic, partition, offset, timestamp, timestampType]
   ```
   
   #### Why this incompatibility bug didn't fail the `KafkaMicroBatchV2SourceSuite` when it merged?
   
   Because the UT `default config of includeHeader doesn't break the existing query from Spark 2.4` didn't test the scenario of duplicating and check the answer.
   Although the UT uses the checkpoint written by version 2.4.3 and streaming duplicate operation, it just wants to prove that the new header(added in SPARK-23539) doesn't break the original checkpoint file. 


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441489165



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -143,6 +145,16 @@ case class StateStoreCustomSumMetric(name: String, desc: String) extends StateSt
 case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
 case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected in state store.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The streaming query failed by state format invalidation. " +
+    "The following reasons may cause this: 1. An old Spark version wrote the checkpoint that is " +
+    "incompatible with the current one; 2. Broken checkpoint files; 3. The query is changed " +
+    "among restart. For the first case, you can try to restart the application without " +

Review comment:
       `For the first case`: I think it's for the cases?




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645716243






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644028150


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/124048/
   Test FAILed.


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r463426200



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1237,6 +1237,16 @@ object SQLConf {
       .intConf
       .createWithDefault(10)
 
+  val STATE_STORE_FORMAT_VALIDATION_ENABLED =
+    buildConf("spark.sql.streaming.stateStore.formatValidation.enabled")
+      .internal()
+      .doc("When true, check if the UnsafeRow from the state store is valid or not when running " +

Review comment:
       Sure, will submit a follow-up PR today.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645429724






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638119689






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638119689


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638077280






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637867836






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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638985570


   @HeartSaVioR After taking a further look. Instead of dealing with the iterator, how about adding the invalidation for all state store operations in `StateStoreProvider`? Since we can get the key/value row during load map. WDYT?


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638179559


   **[Test build #123475 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123475/testReport)** for PR 28707 at commit [`ee048bc`](https://github.com/apache/spark/commit/ee048bc547c33ae98c102b64fcecde7ad5629ac1).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637926400


   And personally I'd rather do the check in StateStore with additional overhead of reading "a" row in prior to achieve the same in all stateful operations. 
   
   ```
     /** Get or create a store associated with the id. */
     def get(
         storeProviderId: StateStoreProviderId,
         keySchema: StructType,
         valueSchema: StructType,
         indexOrdinal: Option[Int],
         version: Long,
         storeConf: StateStoreConf,
         hadoopConf: Configuration): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         startMaintenanceIfNeeded()
         val provider = loadedProviders.getOrElseUpdate(
           storeProviderId,
           StateStoreProvider.createAndInit(
             storeProviderId.storeId, keySchema, valueSchema, indexOrdinal, storeConf, hadoopConf)
         )
         reportActiveStoreInstance(storeProviderId)
         provider
       }
       val store = storeProvider.getStore(version)
       val iter = store.iterator()
       if (iter.nonEmpty) {
         val rowPair = iter.next()
         val key = rowPair.key
         val value = rowPair.value
         // TODO: validate key with key schema
         // TODO: validate value with value schema
       }
       store
     }
   ```
   
   For streaming aggregations it initializes "two" state stores so the overhead goes to "two" rows, but I don't think the overhead matters much.
   
   


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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-643916110


   cc @maropu @gatorsmile @HeartSaVioR @dongjoon-hyun 
   
   A new regression bug SPARK-31990 was found when investigating the test failure https://github.com/apache/spark/pull/28707#issuecomment-639861273. The root cause is that [this line](https://github.com/apache/spark/pull/28062/files#diff-7a46f10c3cedbf013cf255564d9483cdL2458) in SPARK-31292 made the order of groupCols in Deduplicate changed, and the order changing will break the validation logic here. That is to say, if we don't have this PR, the executor JVM could probably crash, throw a random exception, or even return a wrong answer when using the checkpoint written by the previous version.
   
   So we have 2 related work of this PR:
   
   - [x] **[Block]** Fix and merge the compatibility issue in #28830
   - [x] [Follow-up] Add new test(or modify the current Kafka test) in #28725
   
   ------------------
   ### More detailed analysis:
   The expected order of `Deduplicate.groupCols` in UT KafkaMicroBatchV2SourceSuite is
   ```
   [timestamp, partition, timestampType, key, offset, topic, value]
   ```
   Which is also the order in the checkpoint written by the version before Spark 3.0
   After the changes in SPARK-31292, the groupCols changed to
   ```
   [key, value, topic, partition, offset, timestamp, timestampType]
   ```
   
   #### Why this incompatibility bug didn't fail the `KafkaMicroBatchV2SourceSuite` when it merged?
   
   Because the UT `default config of includeHeader doesn't break the existing query from Spark 2.4` didn't test the scenario of duplicating and check the answer.
   Although the UT uses the checkpoint written by version 2.4.3 and streaming duplicate operation, it just wants to prove that the new header(added in SPARK-23539) doesn't break the original checkpoint file. 


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434418012



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -94,6 +110,28 @@ abstract class StreamingAggregationStateManagerBaseImpl(
     // discard and don't convert values to avoid computation
     store.getRange(None, None).map(_.key)
   }
+
+  override def unsafeRowFormatValidation(row: UnsafeRow, schema: StructType): Unit = {
+    if (checkFormat && SQLConf.get.getConf(
+        SQLConf.STREAMING_STATE_FORMAT_CHECK_ENABLED) && row != null) {
+      if (schema.fields.length != row.numFields) {

Review comment:
       Actually that's the first version I did. Since the checking logic is only used for streaming aggregation query and also depends on the streaming config, I choose to put it in StreamingAggregationStateManager, WDYT?




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645595358






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645456836






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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638522162


   And I think SPARK-27237 doesn't require a sort of "future-proof" which is preferably be done with a thing with risk - it doesn't touch the existing part of checkpoint and simply put the schema information into a new file. If we find a better way to pack the schema information into the checkpoint, we can simply discard/ignore the file or craft a logic to migrate smoothly. No risk on rolling back in future.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644028023


   **[Test build #124048 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124048/testReport)** for PR 28707 at commit [`e3d841c`](https://github.com/apache/spark/commit/e3d841c96a3ac3842e590680c2306d5b15546340).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644028139


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645433621


   **[Test build #124171 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124171/testReport)** for PR 28707 at commit [`fd74ff9`](https://github.com/apache/spark/commit/fd74ff9c337d06f4cb4ccfc638d837b5ea3d0e11).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645307056






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434423688



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_AGGREGATION_STATE_FORMAT_CHECK_ENABLED =
+    buildConf("spark.sql.streaming.aggregationStateFormatCheck.enabled")
+      .doc("Whether to detect a streaming aggregation query may try to use an invalid UnsafeRow " +
+        "in the state store.")
+      .version("3.1.0")
+      .internal()

Review comment:
       we usually put `internal()` right after `buildConf(...)`




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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441489537



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -143,6 +145,16 @@ case class StateStoreCustomSumMetric(name: String, desc: String) extends StateSt
 case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
 case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected in state store.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The streaming query failed by state format invalidation. " +

Review comment:
       Does it have  to be `SparkException`?




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639813262






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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637992953


   I think this PR and SPARK-27237 are orthogonal, and we should have both. SPARK-27237 is a bit hard to be merged as it changes the checkpoint. We may need more reviews to see if it's future proof (e.g. when we want to support schema evolution of the state store).


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639873447


   Sorry my comment was edited so you may be missed the content, but it is also a sort of pointing out for "pinpointing" - do you think your approach works with other state store providers as well? The root cause isn't bound to the implementation of state store provider but this patch is only addressing HDFS state store provider.
   
   I guess you're trying to find how it can be done less frequently, first time the state is loaded from the file, which is optimal. While I think it can be even done without binding to the state store provider implementation if we really need it, have we measured the actual overhead? If the overhead turns out to be trivial then it won't be matter we run validation check for each batch. It sounds to be sub-optimal, but the overhead would be trivial.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638064669


   **[Test build #123475 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123475/testReport)** for PR 28707 at commit [`ee048bc`](https://github.com/apache/spark/commit/ee048bc547c33ae98c102b64fcecde7ad5629ac1).


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645306547


   **[Test build #124166 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124166/testReport)** for PR 28707 at commit [`01007fb`](https://github.com/apache/spark/commit/01007fb9f03c003bfc00d2e2358c9029b83f16e6).


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637926400


   And personally I'd rather do the check in StateStore with additional overhead of reading "a" row in prior to achieve the same in all stateful operations. 
   
   ```
     /** Get or create a store associated with the id. */
     def get(
         storeProviderId: StateStoreProviderId,
         keySchema: StructType,
         valueSchema: StructType,
         indexOrdinal: Option[Int],
         version: Long,
         storeConf: StateStoreConf,
         hadoopConf: Configuration): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         startMaintenanceIfNeeded()
         val provider = loadedProviders.getOrElseUpdate(
           storeProviderId,
           StateStoreProvider.createAndInit(
             storeProviderId.storeId, keySchema, valueSchema, indexOrdinal, storeConf, hadoopConf)
         )
         reportActiveStoreInstance(storeProviderId)
         provider
       }
       val store = storeProvider.getStore(version)
       val iter = store.iterator()
       if (iter.nonEmpty) {
         val rowPair = iter.next()
         val key = rowPair.key
         val value = rowPair.value
         // TODO: validate key with key schema
         // TODO: validate value with value schema
       }
       store
     }
   ```
   
   For streaming aggregations it initializes "two" state stores so the overhead goes to "two" rows, but I don't think the overhead matters much.
   
   If we really concern about the overhead of making additional "iterator" or do the validation on early phase (where it might be possible the state store may not be accessed), just have a StateStore wrapper wrapping `store` and do the same - only validate once for the first "get". In either way, we never need to restrict the functionality to the streaming aggregation.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645307056






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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441618110



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -143,6 +145,16 @@ case class StateStoreCustomSumMetric(name: String, desc: String) extends StateSt
 case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
 case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected in state store.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The streaming query failed by state format invalidation. " +

Review comment:
       No, change it to RuntimeException. Done in fd74ff9.




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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637733113


   cc @zsxwing @rednaxelafx @cloud-fan @HeartSaVioR 


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



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


[GitHub] [spark] cloud-fan edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644635176


   It seems we need to change the state store API to allow Spark to do validation easily. I'm OK to put the validation logic in the state store implementation for now, and figure out how to change the state store API later.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638180161


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639239790


   My alternative with wrapping state store is something like below:
   
   ```
     class RowValidatingStateStore(
         underlying: StateStore,
         keyType: Seq[DataType],
         valueType: Seq[DataType]) extends StateStore {
       private var isValidated = false
   
       override def get(key: UnsafeRow): UnsafeRow = {
         val value = underlying.get(key)
         if (!isValidated) {
           validateRow(value)
           isValidated = true
         }
         value
       }
   
       override def id: StateStoreId = underlying.id
       override def version: Long = underlying.version
       override def put(key: UnsafeRow, value: UnsafeRow): Unit = underlying.put(key, value)
       override def remove(key: UnsafeRow): Unit = underlying.remove(key)
       override def commit(): Long = underlying.commit()
       override def abort(): Unit = underlying.abort()
       override def iterator(): Iterator[UnsafeRowPair] = underlying.iterator()
       override def metrics: StateStoreMetrics = underlying.metrics
       override def hasCommitted: Boolean = underlying.hasCommitted
   
       private def validateRow(row: UnsafeRow): Unit = {
         // TODO: call util method with row and schema to validate
       }
     }
   
     def get(...): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         ...
       }
       // TODO: add if statement to see whether it should wrap state store or not
       new RowValidatingStateStore(storeProvider.getStore(version, keySchema, valueSchema))
     }
   ```
   
   The example code only checks in get operation, which is insufficient to check "key" row in state. That said, iterator approach still provides more possibility of validation, though the validation of unsafe row itself doesn't have enough coverage of checking various incompatibility issues (Definitely we should have another guards as well) so that's a sort of OK to only cover value side.


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434241479



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_STATE_FORMAT_CHECK_ENABLED =

Review comment:
       BTW should we have configuration for this, given that this only does essential check which all rows must have been passed?




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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-643916110


   A new regression bug SPARK-31990 was found when investigating the test failure https://github.com/apache/spark/pull/28707#issuecomment-639861273. The root cause is that [this line](https://github.com/apache/spark/pull/28062/files#diff-7a46f10c3cedbf013cf255564d9483cdL2458) in SPARK-31292 made the order of groupCols in Deduplicate changed, and the order changing will break the validation logic here. That is to say, if we don't have this PR, the executor JVM could probably crash, throw a random exception, or even return a wrong answer when using the checkpoint written by the previous version.
   
   So we have 2 related work of this PR:
   
   - [ ] Fix and merge the compatibility issue in #28830
   - [ ] Add new test(or modify the current Kafka test) in #28725
   
   ------------------
   ### More detailed analysis:
   The expected order of `Deduplicate.groupCols` in UT KafkaMicroBatchV2SourceSuite is
   ```
   [timestamp, partition, timestampType, key, offset, topic, value]
   ```
   After the changes in SPARK-31292, the groupCols changed to
   ```
   [key, value, topic, partition, offset, timestamp, timestampType]
   ```
   
   #### Why this incompatibility bug didn't fail the `KafkaMicroBatchV2SourceSuite` when it merged?
   
   Because the UT `default config of includeHeader doesn't break the existing query from Spark 2.4` didn't test the scenario of duplicating and check the answer.
   Although the UT uses the checkpoint written by version 2.4.3 and streaming duplicate operation, it just wants to prove that the new header(added in SPARK-23539) doesn't break the original checkpoint file. 


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639812505


   **[Test build #123580 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123580/testReport)** for PR 28707 at commit [`7a5e09a`](https://github.com/apache/spark/commit/7a5e09a3d52cc6fa0c5aad0aa1e3c84878afe656).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644004386






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645804908






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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441618274



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
##########
@@ -259,6 +259,7 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit
   @volatile private var storeConf: StateStoreConf = _
   @volatile private var hadoopConf: Configuration = _
   @volatile private var numberOfVersionsToRetainInMemory: Int = _
+  @volatile private var isValidated = false

Review comment:
       Thanks, add the TODO in fd74ff9.




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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-643916110


   cc @maropu @gatorsmile @HeartSaVioR @dongjoon-hyun 
   
   A new regression bug SPARK-31990 was found when investigating the test failure https://github.com/apache/spark/pull/28707#issuecomment-639861273. The root cause is that [this line](https://github.com/apache/spark/pull/28062/files#diff-7a46f10c3cedbf013cf255564d9483cdL2458) in SPARK-31292 made the order of groupCols in Deduplicate changed, and the order changing will break the validation logic here. That is to say, if we don't have this PR, the executor JVM could probably crash, throw a random exception, or even return a wrong answer when using the checkpoint written by the previous version.
   
   So we have 2 related work of this PR:
   
   - [x] **[Block]** Fix and merge the compatibility issue in #28830
   - [ ] [Follow-up] Add new test(or modify the current Kafka test) in #28725
   
   ------------------
   ### More detailed analysis:
   The expected order of `Deduplicate.groupCols` in UT KafkaMicroBatchV2SourceSuite is
   ```
   [timestamp, partition, timestampType, key, offset, topic, value]
   ```
   Which is also the order in the checkpoint written by the version before Spark 3.0
   After the changes in SPARK-31292, the groupCols changed to
   ```
   [key, value, topic, partition, offset, timestamp, timestampType]
   ```
   
   #### Why this incompatibility bug didn't fail the `KafkaMicroBatchV2SourceSuite` when it merged?
   
   Because the UT `default config of includeHeader doesn't break the existing query from Spark 2.4` didn't test the scenario of duplicating and check the answer.
   Although the UT uses the checkpoint written by version 2.4.3 and streaming duplicate operation, it just wants to prove that the new header(added in SPARK-23539) doesn't break the original checkpoint file. 


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639861480


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123580/
   Test FAILed.


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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644020172


   ```
   Sorry my comment was edited so you may be missed the content, but it is also a sort of pointing out for "pinpointing" - do you think your approach works with other state store providers as well?
   ```
   @HeartSaVioR I totally agree with your approach is clean enough for all the state store provider, that's also what I want to achieve. But as you may find that the new regression bug SPARK-31990 is found by key validation, that's why we can't directly skip it. Anyway, super thanks for the proposal, let's try to find another way together which matches all the requirements, without changing the provider API.


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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-646450708


   thanks, merging to master! (I think this patch is too big to backport)


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637735695






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644004386






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441674969



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+
+object UnsafeRowUtils {
+
+  /**
+   * Use the following rules to check the integrity of the UnsafeRow:
+   * - schema.fields.length == row.numFields should always be true
+   * - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < row.getSizeInBytes should always be
+   *   true if the expectedSchema contains at least one field.
+   * - For variable-length fields: if null bit says it's null then don't do anything, else extract
+   *   offset and size:
+   *   1) 0 <= size < row.getSizeInBytes should always be true. We can be even more precise than
+   *      this, where the upper bound of size can only be as big as the variable length part of
+   *      the row.
+   *   2) offset should be >= fixed sized part of the row.
+   *   3) offset + size should be within the row bounds.
+   * - For fixed-length fields that are narrower than 8 bytes (boolean/byte/short/int/float), if
+   *   null bit says it's null then don't do anything, else:
+   *     check if the unused bits in the field are all zeros. The UnsafeRowWriter's write() methods
+   *     make this guarantee.
+   * - Check the total length of the row.
+   */
+  def validateStructuralIntegrity(row: UnsafeRow, expectedSchema: StructType): Boolean = {
+    if (expectedSchema.fields.length != row.numFields) {
+      return false
+    }
+    val bitSetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields)
+    val rowSizeInBytes = row.getSizeInBytes
+    if (expectedSchema.fields.length > 0 && bitSetWidthInBytes >= rowSizeInBytes) {
+      return false
+    }
+    var varLenFieldsSizeInBytes = 0
+    expectedSchema.fields.zipWithIndex.foreach {
+      case (field, index) if !UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        val offsetAndSize = row.getLong(index)
+        val offset = (offsetAndSize >> 32).toInt
+        val size = offsetAndSize.toInt
+        if (size < 0 ||
+            offset < bitSetWidthInBytes + 8 * row.numFields || offset + size > rowSizeInBytes) {
+          return false
+        }
+        varLenFieldsSizeInBytes += size
+      case (field, index) if UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        field.dataType match {
+          case BooleanType =>
+            if ((row.getLong(index) >> 1) != 0L) return false
+          case ByteType =>
+            if ((row.getLong(index) >> 8) != 0L) return false
+          case ShortType =>
+            if ((row.getLong(index) >> 16) != 0L) return false
+          case IntegerType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case FloatType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case _ =>
+        }
+      case (field, index) if field.dataType == NullType =>

Review comment:
       I mean `case (field, index) if row.isNullAt(index)`
   
   `UnsafeRowWriter` will zero-out the offsetAndRegion if the field value is null.




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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638626662


   Yea we need this PR to unblock backporting SPARK-28067 to 3.0.
   
   > the mechanism is a validation of the UnsafeRow which can be applied to all stateful operations.
   
   What are other stateful operations that use unsafe row? I think we can apply the check everywhere.
   
   > Unless we have a chance to bring a show-stopper bug in the validation logic
   
   This is something we don't know. Adding a flag seems safer.
   
   > it doesn't touch the existing part of checkpoint and simply put the schema information into a new file.
   
   I'm not saying we shouldn't merge it. I just want to prioritize this PR so that we may be able to include sum correctness bug in 3.0.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637865791


   **[Test build #123447 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123447/testReport)** for PR 28707 at commit [`4d14961`](https://github.com/apache/spark/commit/4d14961499a23498c8af0fabd842a1ed46e6cfc7).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644003732


   **[Test build #124048 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124048/testReport)** for PR 28707 at commit [`e3d841c`](https://github.com/apache/spark/commit/e3d841c96a3ac3842e590680c2306d5b15546340).


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441843015



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1237,6 +1237,15 @@ object SQLConf {
       .intConf
       .createWithDefault(10)
 
+  val STATE_STORE_FORMAT_VALIDATION_ENABLED =
+    buildConf("spark.sql.streaming.stateStore.formatValidation.enabled")
+      .internal()
+      .doc("When true, check if the UnsafeRow from the state store is valid or not when running " +
+        "streaming queries. This can happen if the state store format has been changed.")

Review comment:
       I'd explicitly describe that the feature is only effective in built-in HDFS state store provider.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639861463






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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434420491



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_STATE_FORMAT_CHECK_ENABLED =

Review comment:
       Thanks, rename it in ee048bc. Considering it's an extra checking and still have overhead, I keep the feature flag for safety.




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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434429391



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_AGGREGATION_STATE_FORMAT_CHECK_ENABLED =
+    buildConf("spark.sql.streaming.aggregationStateFormatCheck.enabled")
+      .doc("Whether to detect a streaming aggregation query may try to use an invalid UnsafeRow " +

Review comment:
       Thanks, rephrase in 10a7980.




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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638179692


   @rednaxelafx Great thanks for the detailed comment and guidance. I'm addressing these comments.


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639873447


   Sorry my comment was edited so you may be missed the content, but it is also a sort of pointing out for "pinpointing" - do you think your approach works with other state store providers as well? The root cause isn't bound to the implementation of state store provider but this patch is only addressing HDFS state store provider.
   
   I guess you're trying to find how it can be done less frequently, first time the state is loaded from the file, which is optimal. While I think it can be even done without binding to the state store provider implementation if we really need it (check only once when the provider instance is created), have we measured the actual overhead? If the overhead turns out to be trivial then it won't be matter we run validation check for each batch. It sounds to be sub-optimal, but the overhead would be trivial.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645595358






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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638179692


   @rednaxelafx Great thanks for the detailed comment and guidance. I'm addressing these comments.
   
   `How about put the validation code in a new object UnsafeRowUtils?`
   
   Sure, a separate utils object makes more sense as we want it to be a general validation logic.


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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638064669


   **[Test build #123475 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123475/testReport)** for PR 28707 at commit [`ee048bc`](https://github.com/apache/spark/commit/ee048bc547c33ae98c102b64fcecde7ad5629ac1).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644028139






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638076637


   **[Test build #123477 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123477/testReport)** for PR 28707 at commit [`10a7980`](https://github.com/apache/spark/commit/10a7980b332c57e77f022fced1cba8768c8e4ff6).


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638520229


   Will this be included to Spark 3.0.0? If this is to unblock SPARK-28067 to be included to Spark 3.0.0 then it's OK to consider this first, but if this plans to go to Spark 3.1 then I'm not sure about the priority - are all of you aware that the PR for SPARK-27237 was submitted more than a year ago, and still be considered as later?
   
   I still don't get why the proposal is restricting its usage to streaming aggregation, whereas the mechanism is validation of the UnsafeRow which can be applied to all stateful operations. Let's not to pinpoint the problem we've just seen.
   
   Also from my side the overhead of the validation logic looks to be trivial compared to the operations stateful operators will take - we don't do the validation for all rows, even don't sample, just the first one. Unless we have a chance to bring a show-stopper bug in the validation logic (so that we need to provide the way to disable the validation), I'm not seeing the needs of new configuration.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639873447


   Sorry my comment was edited so you may be missed the content, but it is also a sort of pointing out of "pinpoint" - do you think your approach works with other state store providers as well? The root cause isn't bound to the implementation of state store provider but this patch is only addressing in HDFS state store provider.
   
   I guess you're trying to find how it can be done less frequently, first time the state is loaded from the file, which is optimal. While I think it can be even done without binding to the state store provider implementation if we really need it, have we measured the actual overhead? If the overhead turns out to be trivial then it won't be matter we run validation check for each batch. It sounds to be sub-optimal, but the overhead would be trivial.


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434419336



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -77,13 +82,24 @@ object StreamingAggregationStateManager extends Logging {
   }
 }
 
+/**
+ * An exception thrown when an invalid UnsafeRow is detected.
+ */
+class InvalidUnsafeRowException
+  extends SparkException("The UnsafeRow format is invalid. This may happen when using the old " +
+    "version or broken checkpoint file. To resolve this problem, you can try to restart the " +

Review comment:
       Thanks for the comments, I rephrase the error message to make it clearer. Yep, there are several ways that can lead to the invalid format and we need to list them all. Done in ee048bc




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639861463


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] xuanyuanking commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-646497299


   Thanks all for reviewing!
   I'll review #24173 as the next step for schema validation.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645456836






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645429724






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



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


[GitHub] [spark] cloud-fan closed pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #28707:
URL: https://github.com/apache/spark/pull/28707


   


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645716243






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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639861273


   **[Test build #123580 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123580/testReport)** for PR 28707 at commit [`7a5e09a`](https://github.com/apache/spark/commit/7a5e09a3d52cc6fa0c5aad0aa1e3c84878afe656).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639809051






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637867836






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638180161






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638077280






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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645455712


   **[Test build #124166 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124166/testReport)** for PR 28707 at commit [`01007fb`](https://github.com/apache/spark/commit/01007fb9f03c003bfc00d2e2358c9029b83f16e6).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class StateStoreConf(`


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639813262






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639809051






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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638119541


   **[Test build #123477 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123477/testReport)** for PR 28707 at commit [`10a7980`](https://github.com/apache/spark/commit/10a7980b332c57e77f022fced1cba8768c8e4ff6).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434429538



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1545,6 +1545,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STREAMING_AGGREGATION_STATE_FORMAT_CHECK_ENABLED =
+    buildConf("spark.sql.streaming.aggregationStateFormatCheck.enabled")
+      .doc("Whether to detect a streaming aggregation query may try to use an invalid UnsafeRow " +
+        "in the state store.")
+      .version("3.1.0")
+      .internal()

Review comment:
       Thanks, done in 10a7980.




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



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


[GitHub] [spark] skambha commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
skambha commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639187545


   > @skambha You can check the integrated tests in #28725. If we delete the validation, we'll get a NPE for [this test](https://github.com/apache/spark/pull/28725/files#diff-492f0d70824a58ef2ea94a54dc6f9707R79), and get an assertion in the unsafe row for [this test](https://github.com/apache/spark/pull/28725/files#diff-492f0d70824a58ef2ea94a54dc6f9707R185). That is to say, we will get random failures during reusing the checkpoint written by the old Spark version.
   
   Thanks for adding the test. 


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-644003732


   **[Test build #124048 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124048/testReport)** for PR 28707 at commit [`e3d841c`](https://github.com/apache/spark/commit/e3d841c96a3ac3842e590680c2306d5b15546340).


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-639200645


   > @HeartSaVioR After taking a further look. Instead of dealing with the iterator, how about adding the invalidation for all state store operations in StateStoreProvider? Since we can get the key/value row during load map. WDYT?
   
   It would be nice to see the proposed change by code to avoid misunderstanding, like I proposed in previous comment. (anything including commit in your fork or text comment is OK) I'll try out my alternative (wrapping State Store) and show the code change. Thanks!
   
   EDIT: Please deal with interface whenever possible - there're different implementations of state store providers and we should avoid sticking to the specific implementation.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r441487599



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+
+object UnsafeRowUtils {
+
+  /**
+   * Use the following rules to check the integrity of the UnsafeRow:
+   * - schema.fields.length == row.numFields should always be true
+   * - UnsafeRow.calculateBitSetWidthInBytes(row.numFields) < row.getSizeInBytes should always be
+   *   true if the expectedSchema contains at least one field.
+   * - For variable-length fields: if null bit says it's null then don't do anything, else extract
+   *   offset and size:
+   *   1) 0 <= size < row.getSizeInBytes should always be true. We can be even more precise than
+   *      this, where the upper bound of size can only be as big as the variable length part of
+   *      the row.
+   *   2) offset should be >= fixed sized part of the row.
+   *   3) offset + size should be within the row bounds.
+   * - For fixed-length fields that are narrower than 8 bytes (boolean/byte/short/int/float), if
+   *   null bit says it's null then don't do anything, else:
+   *     check if the unused bits in the field are all zeros. The UnsafeRowWriter's write() methods
+   *     make this guarantee.
+   * - Check the total length of the row.
+   */
+  def validateStructuralIntegrity(row: UnsafeRow, expectedSchema: StructType): Boolean = {
+    if (expectedSchema.fields.length != row.numFields) {
+      return false
+    }
+    val bitSetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields)
+    val rowSizeInBytes = row.getSizeInBytes
+    if (expectedSchema.fields.length > 0 && bitSetWidthInBytes >= rowSizeInBytes) {
+      return false
+    }
+    var varLenFieldsSizeInBytes = 0
+    expectedSchema.fields.zipWithIndex.foreach {
+      case (field, index) if !UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        val offsetAndSize = row.getLong(index)
+        val offset = (offsetAndSize >> 32).toInt
+        val size = offsetAndSize.toInt
+        if (size < 0 ||
+            offset < bitSetWidthInBytes + 8 * row.numFields || offset + size > rowSizeInBytes) {
+          return false
+        }
+        varLenFieldsSizeInBytes += size
+      case (field, index) if UnsafeRow.isFixedLength(field.dataType) && !row.isNullAt(index) =>
+        field.dataType match {
+          case BooleanType =>
+            if ((row.getLong(index) >> 1) != 0L) return false
+          case ByteType =>
+            if ((row.getLong(index) >> 8) != 0L) return false
+          case ShortType =>
+            if ((row.getLong(index) >> 16) != 0L) return false
+          case IntegerType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case FloatType =>
+            if ((row.getLong(index) >> 32) != 0L) return false
+          case _ =>
+        }
+      case _ =>

Review comment:
       For null field, we can also check `row.getLong(index) == 0`, as it's also guaranteed by the `UnsafeRowWriter`




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



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


[GitHub] [spark] SparkQA commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-645594379


   **[Test build #124171 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124171/testReport)** for PR 28707 at commit [`fd74ff9`](https://github.com/apache/spark/commit/fd74ff9c337d06f4cb4ccfc638d837b5ea3d0e11).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637735027


   **[Test build #123447 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/123447/testReport)** for PR 28707 at commit [`4d14961`](https://github.com/apache/spark/commit/4d14961499a23498c8af0fabd842a1ed46e6cfc7).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638119697


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/123477/
   Test FAILed.


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



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


[GitHub] [spark] cloud-fan commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-638166050


   > Having a util function that does this check inside of Spark would be very handy for future low-level debugging / investigations.
   
   +1. How about put the validation code in a new object `UnsafeRowUtils`?


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



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


[GitHub] [spark] skambha commented on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
skambha commented on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637904536


   @xuanyuanking , can you please explain how this will fix the issue where we have changed something in the internal implementation of sum in SPARK-28067, how does that affect previous states and what would be the expected behavior.  From a query level, the sum schema is same.  Is the checkpoint storing information that is coming from intermediate states.   Are we storing unsafe rows from the updateExpression/ or the merge phases of aggregation?  


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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #28707:
URL: https://github.com/apache/spark/pull/28707#issuecomment-637926400


   And personally I'd rather do the check in StateStore with additional overhead of reading "a" row in prior to achieve the same in all stateful operations. 
   
   ```
     /** Get or create a store associated with the id. */
     def get(
         storeProviderId: StateStoreProviderId,
         keySchema: StructType,
         valueSchema: StructType,
         indexOrdinal: Option[Int],
         version: Long,
         storeConf: StateStoreConf,
         hadoopConf: Configuration): StateStore = {
       require(version >= 0)
       val storeProvider = loadedProviders.synchronized {
         startMaintenanceIfNeeded()
         val provider = loadedProviders.getOrElseUpdate(
           storeProviderId,
           StateStoreProvider.createAndInit(
             storeProviderId.storeId, keySchema, valueSchema, indexOrdinal, storeConf, hadoopConf)
         )
         reportActiveStoreInstance(storeProviderId)
         provider
       }
       val store = storeProvider.getStore(version)
       val iter = store.iterator()
       if (iter.nonEmpty) {
         val rowPair = iter.next()
         val key = rowPair.key
         val value = rowPair.value
         // TODO: validate key with key schema
         // TODO: validate value with value schema
       }
       store
     }
   ```
   
   For streaming aggregations it initializes "two" state stores so the overhead goes to "two" rows, but I don't think the overhead matters much.
   
   If we really concern about the overhead, just have a StateStore wrapper wrapping `store` and do the same - only validate once for the first "get". In either way, we never need to restrict the functionality to the streaming aggregation.


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



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


[GitHub] [spark] rednaxelafx commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store

Posted by GitBox <gi...@apache.org>.
rednaxelafx commented on a change in pull request #28707:
URL: https://github.com/apache/spark/pull/28707#discussion_r434457124



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -94,6 +110,28 @@ abstract class StreamingAggregationStateManagerBaseImpl(
     // discard and don't convert values to avoid computation
     store.getRange(None, None).map(_.key)
   }
+
+  override def unsafeRowFormatValidation(row: UnsafeRow, schema: StructType): Unit = {
+    if (checkFormat && SQLConf.get.getConf(
+        SQLConf.STREAMING_STATE_FORMAT_CHECK_ENABLED) && row != null) {
+      if (schema.fields.length != row.numFields) {

Review comment:
       I was hoping we could move the core validation logic to either `UnsafeRow` itself, or some sort of `UnsafeRowUtils`, maybe somewhere in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util`.
   
   This util function would either return a boolean indicating passed/failed integrity check, or it could return more details. I'd probably go with the former first. It would not do any conf checks -- that's the caller's responsibility. This utility is useful for debugging low-level stuff in general, and would come in handy in both Spark SQL and Structured Streaming debugging.
   
   Then we can call that util function from here, after checking the confs. And the exception throwing logic can be left here too.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -94,6 +112,28 @@ abstract class StreamingAggregationStateManagerBaseImpl(
     // discard and don't convert values to avoid computation
     store.getRange(None, None).map(_.key)
   }
+
+  override def unsafeRowFormatValidation(row: UnsafeRow, schema: StructType): Unit = {
+    if (checkFormat && SQLConf.get.getConf(
+        SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_CHECK_ENABLED) && row != null) {
+      if (schema.fields.length != row.numFields) {
+        throw new InvalidUnsafeRowException
+      }
+      schema.fields.zipWithIndex
+        .filterNot(field => UnsafeRow.isFixedLength(field._1.dataType)).foreach {
+        case (_, index) =>
+          val offsetAndSize = row.getLong(index)
+          val offset = (offsetAndSize >> 32).toInt
+          val size = offsetAndSize.toInt
+          if (size < 0 ||
+              offset < UnsafeRow.calculateBitSetWidthInBytes(row.numFields) + 8 * row.numFields ||

Review comment:
       `UnsafeRow.calculateBitSetWidthInBytes(row.numFields) + 8 * row.numFields` this part is loop invariant. Please hoist it out of the loop manually here. It's the same kind of logic as `UnsafeRowWriter`'s
   ```java
       this.nullBitsSize = UnsafeRow.calculateBitSetWidthInBytes(numFields);
       this.fixedSize = nullBitsSize + 8 * numFields;
   ```
   We may want to use the same or similar names for the hoisted variables.
   
   `row.getSizeInBytes` on the next line is also loop invariant. Let's also hoist that out.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
##########
@@ -59,6 +61,9 @@ sealed trait StreamingAggregationStateManager extends Serializable {
 
   /** Return an iterator containing all the values in target state store. */
   def values(store: StateStore): Iterator[UnsafeRow]
+
+  /** Check the UnsafeRow format with the expected schema */
+  def unsafeRowFormatValidation(row: UnsafeRow, schema: StructType): Unit

Review comment:
       Nit: I'd like use "verb + noun" names for actions, and "nouns" for properties.
   Here it'd be some form of "validate structural integrity". WDYT?




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



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