[GitHub] [spark] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: 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] cloud-fan commented on a change in pull request #28707: [SPARK-31894][SS] Introduce UnsafeRow format validation for streaming state store
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: us...@infra.apache.org - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org