diff --git a/connector/connect/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/connector/connect/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index 81c5328c9b29b..76d159cfd159a 100644 --- a/connector/connect/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/connector/connect/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -26,4 +26,12 @@ private[spark] object Connect { .intConf .createWithDefault(15002) + val CONNECT_GRPC_INTERCEPTOR_CLASSES = + ConfigBuilder("spark.connect.grpc.interceptor.classes") + .doc( + "Comma separated list of class names that must " + + "implement the io.grpc.ServerInterceptor interface.") + .version("3.4.0") + .stringConf + .createOptional } diff --git a/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterceptorRegistry.scala b/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterceptorRegistry.scala new file mode 100644 index 0000000000000..cddd4b976638d --- /dev/null +++ b/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterceptorRegistry.scala @@ -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 { + + // 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) +} diff --git a/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 7c494e39a69a0..20776a29edab4 100644 --- a/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -192,6 +192,9 @@ object SparkConnectService { .forPort(port) .addService(new SparkConnectService(debugMode)) + // Add all registered interceptors to the server builder. + SparkConnectInterceptorRegistry.chainInterceptors(sb) + // If debug mode is configured, load the ProtoReflection service so that tools like // grpcurl can introspect the API for debugging. if (debugMode) { diff --git a/connector/connect/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala b/connector/connect/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala new file mode 100644 index 0000000000000..bac02ec7af695 --- /dev/null +++ b/connector/connect/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala @@ -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 + } + } + } + +} diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 0cfb6861c776c..2625fc8ffd59d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -76,6 +76,23 @@ "Another instance of this query was just started by a concurrent session." ] }, + "CONNECT" : { + "message" : [ + "Generic Spark Connect error." + ], + "subClass" : { + "INTERCEPTOR_CTOR_MISSING" : { + "message" : [ + "Cannot instantiate GRPC interceptor because is missing a default constructor without arguments." + ] + }, + "INTERCEPTOR_RUNTIME_ERROR" : { + "message" : [ + "Error instantiating GRPC interceptor: " + ] + } + } + }, "CONVERSION_INVALID_INPUT" : { "message" : [ "The value () cannot be converted to because it is malformed. Correct the value as per the syntax, or change its format. Use to tolerate malformed input and return NULL instead." @@ -4276,4 +4293,4 @@ "Not enough memory to build and broadcast the table to all worker nodes. As a workaround, you can either disable broadcast by setting to -1 or increase the spark driver memory by setting to a higher value" ] } -} +} \ No newline at end of file