This is an automated email from the ASF dual-hosted git repository. ruifengz pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 119271d49356 [SPARK-47868][CONNECT] Fix recursion limit error in SparkConnectPlanner and SparkSession 119271d49356 is described below commit 119271d4935605f15c358c52410dc20db40ace86 Author: Tom van Bussel <tom.vanbus...@databricks.com> AuthorDate: Wed Apr 17 07:33:24 2024 +0800 [SPARK-47868][CONNECT] Fix recursion limit error in SparkConnectPlanner and SparkSession <!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html 2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html 3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'. 4. Be sure to keep the PR description updated to reflect all changes. 5. Please write your PR title to summarize what this PR proposes. 6. If possible, provide a concise example to reproduce the issue for a faster review. 7. If you want to add a new configuration, please read the guideline first for naming configurations in 'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'. 8. If you want to add or modify an error type or message, please read the guideline first in 'common/utils/src/main/resources/error/README.md'. --> ### What changes were proposed in this pull request? <!-- Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below. 1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers. 2. If you fix some SQL features, you can provide some references of other DBMSes. 3. If there is design documentation, please add the link. 4. If there is a discussion in the mailing list, please add the link. --> This PR adds a helper function to `ProtoUtils` that calls a proto parser on a byte array with an increased recursion limit. This helper function is used to enhance the `parseFrom` calls in `SparkSession` and `SparkConnectPlanner`. ### Why are the changes needed? <!-- Please clarify why the changes are needed. For instance, 1. If you propose a new API, clarify the use case for a new API. 2. If you fix a bug, you can clarify why it is a bug. --> Otherwise Spark Connect extensions will get the following exception: ``` org.apache.spark.SparkException: grpc_shaded.com.google.protobuf.InvalidProtocolBufferException: Protocol message had too many levels of nesting. May be malicious. Use CodedInputStream.setRecursionLimit() to increase the depth limit. ``` ### Does this PR introduce _any_ user-facing change? <!-- Note that it means *any* user-facing change including all aspects such as the documentation fix. If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible. If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master. If no, write 'No'. --> No ### How was this patch tested? <!-- If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible. If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future. If tests were not added, please describe why they were not added and/or why it was difficult to add. If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks. --> Existing tests to make sure nothing breaks. Manually tested (using an extension that is currently in development) that this solves the exceptions. ### Was this patch authored or co-authored using generative AI tooling? <!-- If generative AI tooling has been used in the process of authoring this patch, please include the phrase: 'Generated-by: ' followed by the name of the tool and its version. If no, write 'No'. Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details. --> No Closes #46075 from tomvanbussel/SPARK-47868. Authored-by: Tom van Bussel <tom.vanbus...@databricks.com> Signed-off-by: Ruifeng Zheng <ruife...@apache.org> --- .../scala/org/apache/spark/sql/SparkSession.scala | 7 +++++- .../sql/connect/client/SparkConnectClient.scala | 10 ++++++++- .../spark/sql/connect/common/ProtoUtils.scala | 25 ++++++++++++++++++++-- .../sql/connect/common/config/ConnectCommon.scala | 3 ++- .../apache/spark/sql/connect/config/Connect.scala | 2 +- .../sql/connect/planner/SparkConnectPlanner.scala | 14 ++++++++---- 6 files changed, 51 insertions(+), 10 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5a2d9bc44c9f..54e29c80c728 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BoxedLongEncoder import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration import org.apache.spark.sql.connect.client.arrow.ArrowSerializer +import org.apache.spark.sql.connect.common.ProtoUtils import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.{CatalogImpl, SqlApiConf} import org.apache.spark.sql.streaming.DataStreamReader @@ -586,9 +587,13 @@ class SparkSession private[sql] ( @DeveloperApi def execute(extension: Array[Byte]): Unit = { + val any = ProtoUtils.parseWithRecursionLimit( + extension, + com.google.protobuf.Any.parser(), + recursionLimit = client.configuration.grpcMaxRecursionLimit) val command = proto.Command .newBuilder() - .setExtension(com.google.protobuf.Any.parseFrom(extension)) + .setExtension(any) .build() execute(command) } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index d9d51c15a880..1e7b4e6574dd 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -566,6 +566,13 @@ object SparkConnectClient { def grpcMaxMessageSize: Int = _configuration.grpcMaxMessageSize + def grpcMaxRecursionLimit(recursionLimit: Int): Builder = { + _configuration = _configuration.copy(grpcMaxRecursionLimit = recursionLimit) + this + } + + def grpcMaxRecursionLimit: Int = _configuration.grpcMaxRecursionLimit + def option(key: String, value: String): Builder = { _configuration = _configuration.copy(metadata = _configuration.metadata + ((key, value))) this @@ -703,7 +710,8 @@ object SparkConnectClient { useReattachableExecute: Boolean = true, interceptors: List[ClientInterceptor] = List.empty, sessionId: Option[String] = None, - grpcMaxMessageSize: Int = ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE) { + grpcMaxMessageSize: Int = ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE, + grpcMaxRecursionLimit: Int = ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) { def userContext: proto.UserContext = { val builder = proto.UserContext.newBuilder() diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala index 96bd06b01535..af07ef11cdf3 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.connect.common -import com.google.protobuf.Message +import com.google.protobuf.{CodedInputStream, InvalidProtocolBufferException, Message, Parser} -private[connect] object ProtoUtils { +private[sql] object ProtoUtils { def abbreviate[T <: Message](message: T, maxStringSize: Int = 1024): T = { abbreviate[T](message, Map("STRING" -> maxStringSize)) } @@ -51,4 +51,25 @@ private[connect] object ProtoUtils { throw new IllegalArgumentException("Spark Connect tag cannot be an empty string.") } } + + def parseWithRecursionLimit[T <: Message]( + bytes: Array[Byte], + parser: Parser[T], + recursionLimit: Int): T = { + val cis = CodedInputStream.newInstance(bytes) + cis.setSizeLimit(Integer.MAX_VALUE) + cis.setRecursionLimit(recursionLimit) + val message = parser.parseFrom(cis) + try { + // If the last tag is 0, it means the message is correctly parsed. + // If the last tag is not 0, it means the message is not correctly + // parsed, and we should throw an exception. + cis.checkLastTagWas(0) + message + } catch { + case e: InvalidProtocolBufferException => + e.setUnfinishedMessage(message) + throw e + } + } } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/config/ConnectCommon.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/config/ConnectCommon.scala index dca65cf905fc..6a7874add463 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/config/ConnectCommon.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/config/ConnectCommon.scala @@ -18,5 +18,6 @@ package org.apache.spark.sql.connect.common.config private[sql] object ConnectCommon { val CONNECT_GRPC_BINDING_PORT: Int = 15002 - val CONNECT_GRPC_MAX_MESSAGE_SIZE: Int = 128 * 1024 * 1024; + val CONNECT_GRPC_MAX_MESSAGE_SIZE: Int = 128 * 1024 * 1024 + val CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT: Int = 1024 } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index e94e86587393..f9d604712420 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -73,7 +73,7 @@ object Connect { |""".stripMargin) .version("3.5.0") .intConf - .createWithDefault(1024) + .createWithDefault(ConnectCommon.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) val CONNECT_SESSION_MANAGER_DEFAULT_SESSION_TIMEOUT = buildStaticConf("spark.connect.session.manager.defaultSessionTimeout") diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 1ef4bbec3e03..fab222146d9f 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -57,8 +57,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendColumns, CoGroup, Coll import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} -import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} -import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidPlanInput, LiteralValueProtoConverter, ProtoUtils, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} +import org.apache.spark.sql.connect.config.Connect.{CONNECT_GRPC_ARROW_MAX_BATCH_SIZE, CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT} import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionHolder, SparkConnectService} import org.apache.spark.sql.connect.utils.MetricGenerator @@ -232,7 +232,10 @@ class SparkConnectPlanner( @DeveloperApi def transformRelation(bytes: Array[Byte]): LogicalPlan = { - transformRelation(proto.Relation.parseFrom(bytes)) + val recursionLimit = session.conf.get(CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) + val relation = + ProtoUtils.parseWithRecursionLimit(bytes, proto.Relation.parser(), recursionLimit) + transformRelation(relation) } private def transformRelationPlugin(extension: ProtoAny): LogicalPlan = { @@ -1512,7 +1515,10 @@ class SparkConnectPlanner( @DeveloperApi def transformExpression(bytes: Array[Byte]): Expression = { - transformExpression(proto.Expression.parseFrom(bytes)) + val recursionLimit = session.conf.get(CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT) + val expression = + ProtoUtils.parseWithRecursionLimit(bytes, proto.Expression.parser(), recursionLimit) + transformExpression(expression) } private def toNamedExpression(expr: Expression): NamedExpression = expr match { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org