This is an automated email from the ASF dual-hosted git repository.

wenchen 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 4ba7ce2136a [SPARK-40857][CONNECT] Enable configurable GPRC 
Interceptors
4ba7ce2136a is described below

commit 4ba7ce2136ab4dcdb5e4b28e52e4c3864f5f8e3f
Author: Martin Grund <martin.gr...@databricks.com>
AuthorDate: Mon Oct 24 23:09:30 2022 +0800

    [SPARK-40857][CONNECT] Enable configurable GPRC Interceptors
    
    ### What changes were proposed in this pull request?
    To be able to modify the incoming requests for the Spark Connect GRPC 
service, for example to be able to translate metadata from the HTTP/2 request 
to values in the proto message the GRPC service needs to be configured using an 
interceptor.
    
    This patch adds two ways to configure interceptors for the GRPC service. 
First, we can now configure interceptors in the 
`SparkConnectInterceptorRegistry` by adding a value to the `interceptorChain` 
like in the example below:
    
    ```
    object SparkConnectInterceptorRegistry {
    
      // Contains the list of configured interceptors.
      private lazy val interceptorChain: Seq[InterceptorBuilder] = Seq(
        interceptor[LoggingInterceptor](classOf[LoggingInterceptor])
      )
      // ...
    }
    ```
    
    The second way to configure interceptors is by configuring them using Spark 
configuration values at startup. Therefore a new config key has been added 
called: `spark.connect.grpc.interceptor.classes`. This config value contains a 
comma-separated list of classes that are added as interceptors to the system.
    
    ```
    ./bin/pyspark --conf 
spark.connect.grpc.interceptor.classes=com.my.important.LoggingInterceptor
    ```
    
    During startup all of the interceptors are added in order to the 
`NettyServerBuilder`.
    
    ```
    // Add all registered interceptors to the server builder.
    SparkConnectInterceptorRegistry.chainInterceptors(sb)
    ```
    
    ### Why are the changes needed?
    Provide a configurable and extensible way to configure interceptors.
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    Unit Tests
    
    Closes #38320 from grundprinzip/SPARK-40857.
    
    Lead-authored-by: Martin Grund <martin.gr...@databricks.com>
    Co-authored-by: Martin Grund <grundprin...@gmail.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../apache/spark/sql/connect/config/Connect.scala  |   8 +
 .../service/SparkConnectInterceptorRegistry.scala  | 109 ++++++++++++++
 .../sql/connect/service/SparkConnectService.scala  |   3 +
 .../connect/service/InterceptorRegistrySuite.scala | 167 +++++++++++++++++++++
 core/src/main/resources/error/error-classes.json   |  19 ++-
 5 files changed, 305 insertions(+), 1 deletion(-)

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 81c5328c9b2..76d159cfd15 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 00000000000..cddd4b97663
--- /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 7c494e39a69..20776a29eda 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 00000000000..bac02ec7af6
--- /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 0f9b665718c..804b95c6550 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 <cls> is missing a 
default constructor without arguments."
+        ]
+      },
+      "INTERCEPTOR_RUNTIME_ERROR" : {
+        "message" : [
+          "Error instantiating GRPC interceptor: <msg>"
+        ]
+      }
+    }
+  },
   "CONVERSION_INVALID_INPUT" : {
     "message" : [
       "The value <str> (<fmt>) cannot be converted to <targetType> because it 
is malformed. Correct the value as per the syntax, or change its format. Use 
<suggestion> to tolerate malformed input and return NULL instead."
@@ -4291,4 +4308,4 @@
       "Not enough memory to build and broadcast the table to all worker nodes. 
As a workaround, you can either disable broadcast by setting 
<autoBroadcastjoinThreshold> to -1 or increase the spark driver memory by 
setting <driverMemory> to a higher value<analyzeTblMsg>"
     ]
   }
-}
+}
\ No newline at end of file


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

Reply via email to