-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-40857] [CONNECT] Enable configurable GPRC Interceptors. #38320
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
b738d09
bdac014
8c12127
c544e3c
c90d235
ebf1e4c
add1e82
e27f118
5952fe3
25ac334
2293318
0817329
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,109 @@ | ||
| /* | ||
| * 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.connect.service | ||
|
|
||
| import java.lang.reflect.InvocationTargetException | ||
|
|
||
| import io.grpc.ServerInterceptor | ||
| import io.grpc.netty.NettyServerBuilder | ||
|
|
||
| import org.apache.spark.{SparkEnv, SparkException} | ||
| import org.apache.spark.sql.connect.config.Connect | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * This object provides a global list of configured interceptors for GRPC. The interceptors are | ||
| * added to the GRPC server in order of their position in the list. Once the statically compiled | ||
| * interceptors are added, dynamically configured interceptors are added. | ||
| */ | ||
| object SparkConnectInterceptorRegistry { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this an API? We should mark
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Or it has to be
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought everything under
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is not .. unless we document so ... Should probably either document it like https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala or explicitly make it private/public.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let's document it, just like what catalyst does. cc @amaliujia
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. makes sense. Let me check places that is intentional to be private or internal API but are documented. I can follow up on it. |
||
|
|
||
| // Contains the list of configured interceptors. | ||
| private lazy val interceptorChain: Seq[InterceptorBuilder] = Seq( | ||
| // Adding a new interceptor at compile time works like the eaxmple below with the dummy | ||
| // interceptor: | ||
| // interceptor[DummyInterceptor](classOf[DummyInterceptor]) | ||
| ) | ||
|
|
||
| /** | ||
| * Given a NettyServerBuilder instance, will chain all interceptors to it in reverse order. | ||
| * @param sb | ||
| */ | ||
| def chainInterceptors(sb: NettyServerBuilder): Unit = { | ||
| interceptorChain.foreach(i => sb.intercept(i())) | ||
| createConfiguredInterceptors().foreach(sb.intercept(_)) | ||
| } | ||
|
|
||
| // Type used to identify the closure responsible to instantiate a ServerInterceptor. | ||
| type InterceptorBuilder = () => ServerInterceptor | ||
|
|
||
| /** | ||
| * Exposed for testing only. | ||
| */ | ||
| def createConfiguredInterceptors(): Seq[ServerInterceptor] = { | ||
| // Check all values from the Spark conf. | ||
| val classes = SparkEnv.get.conf.get(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES) | ||
| if (classes.nonEmpty) { | ||
| classes.get | ||
| .split(",") | ||
| .map(_.trim) | ||
| .filter(_.nonEmpty) | ||
| .map(Utils.classForName[ServerInterceptor](_)) | ||
| .map(createInstance(_)) | ||
| } else { | ||
| Seq.empty | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Creates a new instance of T using the default constructor. | ||
| * @param cls | ||
| * @tparam T | ||
| * @return | ||
| */ | ||
| private def createInstance[T <: ServerInterceptor](cls: Class[T]): ServerInterceptor = { | ||
| val ctorOpt = cls.getConstructors.find(_.getParameterCount == 0) | ||
| if (ctorOpt.isEmpty) { | ||
| throw new SparkException( | ||
| errorClass = "CONNECT.INTERCEPTOR_CTOR_MISSING", | ||
| messageParameters = Map("cls" -> cls.getName), | ||
| cause = null) | ||
| } | ||
| try { | ||
| ctorOpt.get.newInstance().asInstanceOf[T] | ||
| } catch { | ||
| case e: InvocationTargetException => | ||
| throw new SparkException( | ||
| errorClass = "CONNECT.INTERCEPTOR_RUNTIME_ERROR", | ||
| messageParameters = Map("msg" -> e.getTargetException.getMessage), | ||
| cause = e) | ||
| case e: Exception => | ||
| throw new SparkException( | ||
| errorClass = "CONNECT.INTERCEPTOR_RUNTIME_ERROR", | ||
| messageParameters = Map("msg" -> e.getMessage), | ||
| cause = e) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Creates a callable expression that instantiates the configured GPRC interceptor | ||
| * implementation. | ||
| */ | ||
| private def interceptor[T <: ServerInterceptor](cls: Class[T]): InterceptorBuilder = | ||
| () => createInstance(cls) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,167 @@ | ||
| /* | ||
| * 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.connect.service | ||
|
|
||
| import io.grpc.{Metadata, ServerCall, ServerCallHandler, ServerInterceptor} | ||
| import io.grpc.ForwardingServerCallListener.SimpleForwardingServerCallListener | ||
| import io.grpc.netty.NettyServerBuilder | ||
|
|
||
| import org.apache.spark.{SparkEnv, SparkException} | ||
| import org.apache.spark.sql.connect.config.Connect | ||
| import org.apache.spark.sql.test.SharedSparkSession | ||
|
|
||
| /** | ||
| * Used for testing only, does not do anything. | ||
| */ | ||
| class DummyInterceptor extends ServerInterceptor { | ||
| override def interceptCall[ReqT, RespT]( | ||
| call: ServerCall[ReqT, RespT], | ||
| headers: Metadata, | ||
| next: ServerCallHandler[ReqT, RespT]): ServerCall.Listener[ReqT] = { | ||
| val listener = next.startCall(call, headers) | ||
| new SimpleForwardingServerCallListener[ReqT](listener) { | ||
| override def onMessage(message: ReqT): Unit = { | ||
| delegate().onMessage(message) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Used for testing only. | ||
| */ | ||
| class TestingInterceptorNoTrivialCtor(id: Int) extends ServerInterceptor { | ||
| override def interceptCall[ReqT, RespT]( | ||
| call: ServerCall[ReqT, RespT], | ||
| headers: Metadata, | ||
| next: ServerCallHandler[ReqT, RespT]): ServerCall.Listener[ReqT] = { | ||
| val listener = next.startCall(call, headers) | ||
| new SimpleForwardingServerCallListener[ReqT](listener) { | ||
| override def onMessage(message: ReqT): Unit = { | ||
| delegate().onMessage(message) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Used for testing only. | ||
| */ | ||
| class TestingInterceptorInstantiationError extends ServerInterceptor { | ||
| throw new ArrayIndexOutOfBoundsException("Bad Error") | ||
|
|
||
| override def interceptCall[ReqT, RespT]( | ||
| call: ServerCall[ReqT, RespT], | ||
| headers: Metadata, | ||
| next: ServerCallHandler[ReqT, RespT]): ServerCall.Listener[ReqT] = { | ||
| val listener = next.startCall(call, headers) | ||
| new SimpleForwardingServerCallListener[ReqT](listener) { | ||
| override def onMessage(message: ReqT): Unit = { | ||
| delegate().onMessage(message) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class InterceptorRegistrySuite extends SharedSparkSession { | ||
|
|
||
| override def beforeEach(): Unit = { | ||
| if (SparkEnv.get.conf.contains(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES)) { | ||
| SparkEnv.get.conf.remove(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES) | ||
| } | ||
| } | ||
|
|
||
| def withSparkConf(pairs: (String, String)*)(f: => Unit): Unit = { | ||
| val conf = SparkEnv.get.conf | ||
| pairs.foreach { kv => conf.set(kv._1, kv._2) } | ||
| try f | ||
| finally { | ||
| pairs.foreach { kv => conf.remove(kv._1) } | ||
| } | ||
| } | ||
|
|
||
| test("Check that the empty registry works") { | ||
| val sb = NettyServerBuilder.forPort(9999) | ||
| SparkConnectInterceptorRegistry.chainInterceptors(sb) | ||
| } | ||
|
|
||
| test("Test server builder and configured interceptor") { | ||
| withSparkConf( | ||
| Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> | ||
| "org.apache.spark.sql.connect.service.DummyInterceptor") { | ||
| val sb = NettyServerBuilder.forPort(9999) | ||
| SparkConnectInterceptorRegistry.chainInterceptors(sb) | ||
| } | ||
| } | ||
|
|
||
| test("Test server build throws when using bad configured interceptor") { | ||
| withSparkConf( | ||
| Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> | ||
| "org.apache.spark.sql.connect.service.TestingInterceptorNoTrivialCtor") { | ||
| val sb = NettyServerBuilder.forPort(9999) | ||
| assertThrows[SparkException] { | ||
| SparkConnectInterceptorRegistry.chainInterceptors(sb) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("Exception handling for interceptor classes") { | ||
| withSparkConf( | ||
| Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> | ||
| "org.apache.spark.sql.connect.service.TestingInterceptorNoTrivialCtor") { | ||
| assertThrows[SparkException] { | ||
| SparkConnectInterceptorRegistry.createConfiguredInterceptors | ||
| } | ||
| } | ||
|
|
||
| withSparkConf( | ||
| Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> | ||
| "org.apache.spark.sql.connect.service.TestingInterceptorInstantiationError") { | ||
| assertThrows[SparkException] { | ||
| SparkConnectInterceptorRegistry.createConfiguredInterceptors | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("No configured interceptors returns empty list") { | ||
| // Not set. | ||
| assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.isEmpty) | ||
| // Set to empty string | ||
| withSparkConf(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> "") { | ||
| assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.isEmpty) | ||
| } | ||
| } | ||
|
|
||
| test("Configured classes can have multiple entries") { | ||
| withSparkConf( | ||
| Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> | ||
| (" org.apache.spark.sql.connect.service.DummyInterceptor," + | ||
| " org.apache.spark.sql.connect.service.DummyInterceptor ")) { | ||
| assert(SparkConnectInterceptorRegistry.createConfiguredInterceptors.size == 2) | ||
| } | ||
| } | ||
|
|
||
| test("Configured class not found is properly thrown") { | ||
| withSparkConf(Connect.CONNECT_GRPC_INTERCEPTOR_CLASSES.key -> "this.class.does.not.exist") { | ||
| assertThrows[ClassNotFoundException] { | ||
| SparkConnectInterceptorRegistry.createConfiguredInterceptors | ||
| } | ||
| } | ||
| } | ||
|
|
||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just realized that this is under
apach.spark.sql.... we should either move this module out ofsqlor useStaticSQLConfideally.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is in the connect module.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
but it's under
apache.spark.sql.connect. Should probably move it toapache.spark.connect?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
right now, everything in Spark connect is under
apache.spark.sql.connect. Are you proposing a overall package renaming?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I am proposing before it's too late.
Either: if we target to cover other components too, should probably rename them before it's too late. For PySpark too, should probably move it from
pyspark.sql.connect.DataFrame->pyspark.connect.sql.DataFrame.Or, use
StaticSQLConfsince we're in SQL package. We're doing this in Hive thirft server, Hive modules, etc.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Then we should better use
StaticSQLConforSQLConfinstead ofSparkConf.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My point is that it's mixed. It's in SQL package but the configuration being used is
SparkConf. The configuration name doesn't follow it either.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
StaticSQLConf and SQLConf are in the SQL module, it's weird to add spark connect configs there...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's what Hive thriftserver (separate module) dose. Avro (separate module) and Kafka (separate module for Structured Streaming) do. Pandas API on Spark also leverages runtime configurations via SparkSession under the hood instead of SparkConf.
This is weird that it's a SQL thing uses SQL package namespace but it doesn't use
SQLConf.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not sure what's the benefit of doing so. Config definition can be anywhere and we can still use SQLConf to access it. e.g.
SQLConf.get.getConf(CONNECT_GRPC_INTERCEPTOR_CLASSES).