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

2020-07-31 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-17 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-03 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-03 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-03 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-03 Thread GitBox


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:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@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

2020-06-03 Thread GitBox


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:
us...@infra.apache.org



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