- 
                Notifications
    
You must be signed in to change notification settings  - Fork 28.9k
 
[SPARK-51156][CONNECT] Provide a basic authentication token when running Spark Connect server locally #49880
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
          
     Closed
      
      
    
  
     Closed
                    Changes from all commits
      Commits
    
    
            Show all changes
          
          
            2 commits
          
        
        Select commit
          Hold shift + click to select a range
      
      
    File filter
Filter by extension
Conversations
          Failed to load comments.   
        
        
          
      Loading
        
  Jump to
        
          Jump to file
        
      
      
          Failed to load files.   
        
        
          
      Loading
        
  Diff view
Diff view
There are no files selected for viewing
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              | Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -48,6 +48,7 @@ import org.apache.spark.sql.connect.ColumnNodeToProtoConverter.toLiteral | |
| import org.apache.spark.sql.connect.client.{ClassFinder, CloseableIterator, 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.config.ConnectCommon | ||
| import org.apache.spark.sql.internal.{SessionState, SharedState, SqlApiConf} | ||
| import org.apache.spark.sql.sources.BaseRelation | ||
| import org.apache.spark.sql.types.StructType | ||
| 
          
            
          
           | 
    @@ -627,6 +628,17 @@ class SparkSession private[sql] ( | |
| } | ||
| allocator.close() | ||
| SparkSession.onSessionClose(this) | ||
| SparkSession.server.synchronized { | ||
| if (SparkSession.server.isDefined) { | ||
| // When local mode is in use, follow the regular Spark session's | ||
| // behavior by terminating the Spark Connect server, | ||
| // meaning that you can stop local mode, and restart the Spark Connect | ||
| // client with a different remote address. | ||
| new ProcessBuilder(SparkSession.maybeConnectStopScript.get.toString) | ||
| .start() | ||
| SparkSession.server = None | ||
| } | ||
| } | ||
| } | ||
| 
     | 
||
| /** @inheritdoc */ | ||
| 
          
            
          
           | 
    @@ -679,6 +691,10 @@ object SparkSession extends SparkSessionCompanion with Logging { | |
| private val MAX_CACHED_SESSIONS = 100 | ||
| private val planIdGenerator = new AtomicLong | ||
| private var server: Option[Process] = None | ||
| private val maybeConnectStartScript = | ||
| Option(System.getenv("SPARK_HOME")).map(Paths.get(_, "sbin", "start-connect-server.sh")) | ||
| private val maybeConnectStopScript = | ||
| Option(System.getenv("SPARK_HOME")).map(Paths.get(_, "sbin", "stop-connect-server.sh")) | ||
| private[sql] val sparkOptions = sys.props.filter { p => | ||
| p._1.startsWith("spark.") && p._2.nonEmpty | ||
| }.toMap | ||
| 
          
            
          
           | 
    @@ -712,37 +728,43 @@ object SparkSession extends SparkSessionCompanion with Logging { | |
| } | ||
| } | ||
| 
     | 
||
| val maybeConnectScript = | ||
| Option(System.getenv("SPARK_HOME")).map(Paths.get(_, "sbin", "start-connect-server.sh")) | ||
| 
     | 
||
| if (server.isEmpty && | ||
| (remoteString.exists(_.startsWith("local")) || | ||
| (remoteString.isDefined && isAPIModeConnect)) && | ||
| maybeConnectScript.exists(Files.exists(_))) { | ||
| server = Some { | ||
| val args = | ||
| Seq(maybeConnectScript.get.toString, "--master", remoteString.get) ++ sparkOptions | ||
| .filter(p => !p._1.startsWith("spark.remote")) | ||
| .flatMap { case (k, v) => Seq("--conf", s"$k=$v") } | ||
| val pb = new ProcessBuilder(args: _*) | ||
| // So don't exclude spark-sql jar in classpath | ||
| pb.environment().remove(SparkConnectClient.SPARK_REMOTE) | ||
| pb.start() | ||
| } | ||
| 
     | 
||
| // Let the server start. We will directly request to set the configurations | ||
| // and this sleep makes less noisy with retries. | ||
| Thread.sleep(2000L) | ||
| System.setProperty("spark.remote", "sc://localhost") | ||
| 
     | 
||
| // scalastyle:off runtimeaddshutdownhook | ||
| Runtime.getRuntime.addShutdownHook(new Thread() { | ||
| override def run(): Unit = if (server.isDefined) { | ||
| new ProcessBuilder(maybeConnectScript.get.toString) | ||
| .start() | ||
| server.synchronized { | ||
| 
         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. Just added a   | 
||
| if (server.isEmpty && | ||
| (remoteString.exists(_.startsWith("local")) || | ||
| (remoteString.isDefined && isAPIModeConnect)) && | ||
| maybeConnectStartScript.exists(Files.exists(_))) { | ||
| server = Some { | ||
| val args = | ||
| Seq( | ||
| maybeConnectStartScript.get.toString, | ||
| "--master", | ||
| remoteString.get) ++ sparkOptions | ||
| .filter(p => !p._1.startsWith("spark.remote")) | ||
| .flatMap { case (k, v) => Seq("--conf", s"$k=$v") } | ||
| val pb = new ProcessBuilder(args: _*) | ||
| // So don't exclude spark-sql jar in classpath | ||
| pb.environment().remove(SparkConnectClient.SPARK_REMOTE) | ||
| pb.environment() | ||
| .put( | ||
| ConnectCommon.CONNECT_LOCAL_AUTH_TOKEN_ENV_NAME, | ||
| ConnectCommon.CONNECT_LOCAL_AUTH_TOKEN) | ||
| pb.start() | ||
| } | ||
| }) | ||
| // scalastyle:on runtimeaddshutdownhook | ||
| 
     | 
||
| // Let the server start. We will directly request to set the configurations | ||
| // and this sleep makes less noisy with retries. | ||
| Thread.sleep(2000L) | ||
| System.setProperty("spark.remote", "sc://localhost") | ||
| 
     | 
||
| // scalastyle:off runtimeaddshutdownhook | ||
| Runtime.getRuntime.addShutdownHook(new Thread() { | ||
| override def run(): Unit = if (server.isDefined) { | ||
| new ProcessBuilder(maybeConnectStopScript.get.toString) | ||
| .start() | ||
| } | ||
| }) | ||
| // scalastyle:on runtimeaddshutdownhook | ||
| } | ||
| } | ||
| } | ||
| f | ||
| 
          
            
          
           | 
    ||
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
        
          
          
            43 changes: 43 additions & 0 deletions
          
          43 
        
  ...ect/server/src/main/scala/org/apache/spark/sql/connect/service/LocalAuthInterceptor.scala
  
  
      
      
   
        
      
      
    
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              | Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,43 @@ | ||
| /* | ||
| * 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 org.apache.spark.SparkSecurityException | ||
| import org.apache.spark.sql.connect.common.config.ConnectCommon | ||
| 
     | 
||
| /** | ||
| * A gRPC interceptor to check if the header contains token for local authentication. | ||
| */ | ||
| class LocalAuthInterceptor(localToken: String) extends ServerInterceptor { | ||
| override def interceptCall[ReqT, RespT]( | ||
| call: ServerCall[ReqT, RespT], | ||
| headers: Metadata, | ||
| next: ServerCallHandler[ReqT, RespT]): ServerCall.Listener[ReqT] = { | ||
| val token = Option( | ||
| headers.get(Metadata.Key | ||
| .of(ConnectCommon.CONNECT_LOCAL_AUTH_TOKEN_PARAM_NAME, Metadata.ASCII_STRING_MARSHALLER))) | ||
| if (token.isEmpty || token.get != localToken) { | ||
| throw new SparkSecurityException( | ||
| errorClass = "_LEGACY_ERROR_TEMP_3303", | ||
| messageParameters = Map.empty) | ||
| } | ||
| next.startCall(call, headers) | ||
| } | ||
| } | 
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              
        
          
          
            34 changes: 34 additions & 0 deletions
          
          34 
        
  ...r/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectLocalAuthE2ESuite.scala
  
  
      
      
   
        
      
      
    
  
    
      This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
      Learn more about bidirectional Unicode characters
    
  
  
    
              | Original file line number | Diff line number | Diff line change | 
|---|---|---|
| @@ -0,0 +1,34 @@ | ||
| /* | ||
| * 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 org.apache.spark.SparkException | ||
| import org.apache.spark.sql.connect.SparkConnectServerTest | ||
| 
     | 
||
| class SparkConnectLocalAuthE2ESuite extends SparkConnectServerTest { | ||
| override def beforeAll(): Unit = { | ||
| spark.sparkContext.conf.set("spark.testing.token", "invalid") | ||
| super.beforeAll() | ||
| } | ||
| 
     | 
||
| test("Test local authentication") { | ||
| val e = intercept[SparkException] { | ||
| withClient { _ => () } | ||
| } | ||
| e.getCondition == "_LEGACY_ERROR_TEMP_3303" | ||
| } | ||
| } | 
  Add this suggestion to a batch that can be applied as a single commit.
  This suggestion is invalid because no changes were made to the code.
  Suggestions cannot be applied while the pull request is closed.
  Suggestions cannot be applied while viewing a subset of changes.
  Only one suggestion per line can be applied in a batch.
  Add this suggestion to a batch that can be applied as a single commit.
  Applying suggestions on deleted lines is not supported.
  You must change the existing code in this line in order to create a valid suggestion.
  Outdated suggestions cannot be applied.
  This suggestion has been applied or marked resolved.
  Suggestions cannot be applied from pending reviews.
  Suggestions cannot be applied on multi-line comments.
  Suggestions cannot be applied while the pull request is queued to merge.
  Suggestion cannot be applied right now. Please check back later.
  
    
  
    
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.
It's unfortunate that the grpc client tries to force you to use TLS if you want to use call credentials when there's so many workarounds like simply using a different header. Though in this case you could theoretically use
local_channel_credentialsat least on the Python side to use the built-in token mechanismThere 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.
@Kimahriman are you suggesting to use a different header than Authentication? If so, my issue with that is that folks will invariably use this to 'secure' remote connections as well, and in that case having TLS might not be such a bad thing.
We are planning to add UDS support in a follow-up. That should make this less of an issue.
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.
No more the opposite, I assume the custom
local_tokenheader was used here to avoid the TLS requirement, but it's probably ok to allow non-TLS uses of the authorization bearer token header for a local connections, which the Python gRPC client even has support for. Adding the UDS supported would just improve that further. Thelocal_channel_credentialseven supports both of those cases, so you could use that with the authorization header to avoid the TLS requirement for this use case, while still requiring it for remote connections.The "workarounds" are simply that you can still use the authorization header without TLS by using a custom interceptor that injects it in the metadata after the fact. Figured that out while trying to build a custom dynamic proxy for launching cluster deploy mode connect sessions to replace something like Livy.