You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2021/02/25 00:26:55 UTC

[spark] branch branch-3.1 updated: [SPARK-34497][SQL] Fix built-in JDBC connection providers to restore JVM security context changes

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

gurwls223 pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
     new c02c39a  [SPARK-34497][SQL] Fix built-in JDBC connection providers to restore JVM security context changes
c02c39a is described below

commit c02c39a66fe39e4d5f87bc7222ba03a65f1efb7c
Author: Gabor Somogyi <ga...@gmail.com>
AuthorDate: Thu Feb 25 09:25:17 2021 +0900

    [SPARK-34497][SQL] Fix built-in JDBC connection providers to restore JVM security context changes
    
    ### What changes were proposed in this pull request?
    Some of the built-in JDBC connection providers are changing the JVM security context to do the authentication which is fine. The problematic part is that executors can be reused by another query. The following situation leads to incorrect behaviour:
    * Query1 opens JDBC connection and changes JVM security context in Executor1
    * Query2 tries to open JDBC connection but it realizes there is already an entry for that DB type in Executor1
    * Query2 is not changing JVM security context and uses Query1 keytab and principal
    * Query2 fails with authentication error
    
    In this PR I've changed to code such a way that JVM security context is changed all the time but only temporarily until the connection built-up and then rolled back. Since `getConnection` is synchronised with `SecurityConfigurationLock` it ends-up in correct behaviour without any race.
    
    ### Why are the changes needed?
    Incorrect JVM security context handling.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    Existing unit + integration tests.
    
    Closes #31622 from gaborgsomogyi/SPARK-34497.
    
    Authored-by: Gabor Somogyi <ga...@gmail.com>
    Signed-off-by: HyukjinKwon <gu...@apache.org>
    (cherry picked from commit 44eadb943bbcec48e90398731f57a32a967d81bb)
    Signed-off-by: HyukjinKwon <gu...@apache.org>
---
 .../jdbc/connection/ConnectionProvider.scala       | 11 +++++++++-
 .../jdbc/connection/DB2ConnectionProvider.scala    |  9 +-------
 .../jdbc/connection/MSSQLConnectionProvider.scala  | 16 +--------------
 .../connection/MariaDBConnectionProvider.scala     | 17 +--------------
 .../jdbc/connection/OracleConnectionProvider.scala |  9 +-------
 .../connection/PostgresConnectionProvider.scala    |  7 -------
 .../jdbc/connection/SecureConnectionProvider.scala | 20 ++----------------
 .../jdbc/connection/ConnectionProviderSuite.scala  | 24 ++++++++++++++--------
 .../connection/ConnectionProviderSuiteBase.scala   |  8 ++------
 .../connection/DB2ConnectionProviderSuite.scala    |  2 +-
 .../connection/MSSQLConnectionProviderSuite.scala  |  4 ++--
 .../MariaDBConnectionProviderSuite.scala           |  2 +-
 .../connection/OracleConnectionProviderSuite.scala |  2 +-
 .../PostgresConnectionProviderSuite.scala          |  2 +-
 14 files changed, 39 insertions(+), 94 deletions(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
index e81add4..fbc6970 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 import java.sql.{Connection, Driver}
 import java.util.ServiceLoader
+import javax.security.auth.login.Configuration
 
 import scala.collection.mutable
 
@@ -60,7 +61,15 @@ private[jdbc] object ConnectionProvider extends Logging {
       "JDBC connection initiated but not exactly one connection provider found which can handle " +
         s"it. Found active providers: ${filteredProviders.mkString(", ")}")
     SecurityConfigurationLock.synchronized {
-      filteredProviders.head.getConnection(driver, options)
+      // Inside getConnection it's safe to get parent again because SecurityConfigurationLock
+      // makes sure it's untouched
+      val parent = Configuration.getConfiguration
+      try {
+        filteredProviders.head.getConnection(driver, options)
+      } finally {
+        logDebug("Restoring original security configuration")
+        Configuration.setConfiguration(parent)
+      }
     }
   }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala
index 775c3ae..060653c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala
@@ -34,7 +34,7 @@ private[sql] class DB2ConnectionProvider extends SecureConnectionProvider {
 
   override def getConnection(driver: Driver, options: Map[String, String]): Connection = {
     val jdbcOptions = new JDBCOptions(options)
-    setAuthenticationConfigIfNeeded(driver, jdbcOptions)
+    setAuthenticationConfig(driver, jdbcOptions)
     UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab)
       .doAs(
         new PrivilegedExceptionAction[Connection]() {
@@ -52,11 +52,4 @@ private[sql] class DB2ConnectionProvider extends SecureConnectionProvider {
     result.put("KerberosServerPrincipal", options.principal)
     result
   }
-
-  override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = {
-    val (parent, configEntry) = getConfigWithAppEntry(driver, options)
-    if (configEntry == null || configEntry.isEmpty) {
-      setAuthenticationConfig(parent, driver, options)
-    }
-  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala
index e3d3e1a..aa8c922 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala
@@ -61,7 +61,7 @@ private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider {
 
   override def getConnection(driver: Driver, options: Map[String, String]): Connection = {
     val jdbcOptions = new JDBCOptions(options)
-    setAuthenticationConfigIfNeeded(driver, jdbcOptions)
+    setAuthenticationConfig(driver, jdbcOptions)
     UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab)
       .doAs(
         new PrivilegedExceptionAction[Connection]() {
@@ -79,18 +79,4 @@ private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider {
     result.put("authenticationScheme", "JavaKerberos")
     result
   }
-
-  override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = {
-    val (parent, configEntry) = getConfigWithAppEntry(driver, options)
-    /**
-     * Couple of things to mention here (v8.2.2 client):
-     * 1. MS SQL supports JAAS application name configuration
-     * 2. MS SQL sets a default JAAS config if "java.security.auth.login.config" is not set
-     */
-    val entryUsesKeytab = configEntry != null &&
-      configEntry.exists(_.getOptions().get("useKeyTab") == "true")
-    if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) {
-      setAuthenticationConfig(parent, driver, options)
-    }
-  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
index 29a08d0..6a53c66 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
@@ -26,20 +26,5 @@ private[jdbc] class MariaDBConnectionProvider extends SecureConnectionProvider {
 
   override val name: String = "mariadb"
 
-  override def appEntry(driver: Driver, options: JDBCOptions): String =
-    "Krb5ConnectorContext"
-
-  override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = {
-    val (parent, configEntry) = getConfigWithAppEntry(driver, options)
-    /**
-     * Couple of things to mention here (v2.5.4 client):
-     * 1. MariaDB doesn't support JAAS application name configuration
-     * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" is not set
-     */
-    val entryUsesKeytab = configEntry != null &&
-      configEntry.exists(_.getOptions().get("useKeyTab") == "true")
-    if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) {
-      setAuthenticationConfig(parent, driver, options)
-    }
-  }
+  override def appEntry(driver: Driver, options: JDBCOptions): String = "Krb5ConnectorContext"
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala
index 0d43851..ef8d91b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala
@@ -34,7 +34,7 @@ private[sql] class OracleConnectionProvider extends SecureConnectionProvider {
 
   override def getConnection(driver: Driver, options: Map[String, String]): Connection = {
     val jdbcOptions = new JDBCOptions(options)
-    setAuthenticationConfigIfNeeded(driver, jdbcOptions)
+    setAuthenticationConfig(driver, jdbcOptions)
     UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab)
       .doAs(
         new PrivilegedExceptionAction[Connection]() {
@@ -53,11 +53,4 @@ private[sql] class OracleConnectionProvider extends SecureConnectionProvider {
     result.put("oracle.net.authentication_services", "(KERBEROS5)");
     result
   }
-
-  override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = {
-    val (parent, configEntry) = getConfigWithAppEntry(driver, options)
-    if (configEntry == null || configEntry.isEmpty) {
-      setAuthenticationConfig(parent, driver, options)
-    }
-  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala
index f26a11e..ec9ac66 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala
@@ -32,11 +32,4 @@ private[jdbc] class PostgresConnectionProvider extends SecureConnectionProvider
     val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties]
     properties.getProperty("jaasApplicationName", "pgjdbc")
   }
-
-  override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = {
-    val (parent, configEntry) = getConfigWithAppEntry(driver, options)
-    if (configEntry == null || configEntry.isEmpty) {
-      setAuthenticationConfig(parent, driver, options)
-    }
-  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala
index 80c7959..4138c72 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala
@@ -40,7 +40,7 @@ private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionPro
 
   override def getConnection(driver: Driver, options: Map[String, String]): Connection = {
     val jdbcOptions = new JDBCOptions(options)
-    setAuthenticationConfigIfNeeded(driver, jdbcOptions)
+    setAuthenticationConfig(driver, jdbcOptions)
     super.getConnection(driver: Driver, options: Map[String, String])
   }
 
@@ -49,24 +49,8 @@ private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionPro
    */
   def appEntry(driver: Driver, options: JDBCOptions): String
 
-  /**
-   * Sets database specific authentication configuration when needed. If configuration already set
-   * then later calls must be no op. When the global JVM security configuration changed then the
-   * related code parts must be synchronized properly.
-   */
-  def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit
-
-  protected def getConfigWithAppEntry(
-      driver: Driver,
-      options: JDBCOptions): (Configuration, Array[AppConfigurationEntry]) = {
+  private[connection] def setAuthenticationConfig(driver: Driver, options: JDBCOptions) = {
     val parent = Configuration.getConfiguration
-    (parent, parent.getAppConfigurationEntry(appEntry(driver, options)))
-  }
-
-  protected def setAuthenticationConfig(
-      parent: Configuration,
-      driver: Driver,
-      options: JDBCOptions) = {
     val config = new SecureConnectionProvider.JDBCConfiguration(
       parent, appEntry(driver, options), options.keytab, options.principal)
     logDebug("Adding database specific security configuration")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala
index 0e9498b..71b0325 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala
@@ -57,17 +57,23 @@ class ConnectionProviderSuite extends ConnectionProviderSuiteBase with SharedSpa
     val db2AppEntry = db2Provider.appEntry(db2Driver, db2Options)
 
     // Make sure no authentication for the databases are set
-    val oldConfig = Configuration.getConfiguration
-    assert(oldConfig.getAppConfigurationEntry(postgresAppEntry) == null)
-    assert(oldConfig.getAppConfigurationEntry(db2AppEntry) == null)
+    val rootConfig = Configuration.getConfiguration
+    assert(rootConfig.getAppConfigurationEntry(postgresAppEntry) == null)
+    assert(rootConfig.getAppConfigurationEntry(db2AppEntry) == null)
 
-    postgresProvider.setAuthenticationConfigIfNeeded(postgresDriver, postgresOptions)
-    db2Provider.setAuthenticationConfigIfNeeded(db2Driver, db2Options)
+    postgresProvider.setAuthenticationConfig(postgresDriver, postgresOptions)
+    val postgresConfig = Configuration.getConfiguration
+
+    db2Provider.setAuthenticationConfig(db2Driver, db2Options)
+    val db2Config = Configuration.getConfiguration
 
     // Make sure authentication for the databases are set
-    val newConfig = Configuration.getConfiguration
-    assert(oldConfig != newConfig)
-    assert(newConfig.getAppConfigurationEntry(postgresAppEntry) != null)
-    assert(newConfig.getAppConfigurationEntry(db2AppEntry) != null)
+    assert(rootConfig != postgresConfig)
+    assert(rootConfig != db2Config)
+    // The topmost config in the chain is linked with all the subsequent entries
+    assert(db2Config.getAppConfigurationEntry(postgresAppEntry) != null)
+    assert(db2Config.getAppConfigurationEntry(db2AppEntry) != null)
+
+    Configuration.setConfiguration(null)
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala
index a299841..f42b17a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala
@@ -59,16 +59,12 @@ abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndA
     // Make sure no authentication for the database is set
     assert(Configuration.getConfiguration.getAppConfigurationEntry(providerAppEntry) == null)
 
-    // Make sure the first call sets authentication properly
+    // Make sure setAuthenticationConfig call sets authentication properly
     val savedConfig = Configuration.getConfiguration
-    provider.setAuthenticationConfigIfNeeded(driver, options)
+    provider.setAuthenticationConfig(driver, options)
     val config = Configuration.getConfiguration
     assert(savedConfig != config)
     val appEntry = config.getAppConfigurationEntry(providerAppEntry)
     assert(appEntry != null)
-
-    // Make sure a second call is not modifying the existing authentication
-    provider.setAuthenticationConfigIfNeeded(driver, options)
-    assert(config.getAppConfigurationEntry(providerAppEntry) === appEntry)
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala
index 5885af8..895b3d8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 class DB2ConnectionProviderSuite extends ConnectionProviderSuiteBase {
-  test("setAuthenticationConfigIfNeeded must set authentication if not set") {
+  test("setAuthenticationConfig must set authentication all the time") {
     val provider = new DB2ConnectionProvider()
     val driver = registerDriver(provider.driverClass)
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala
index a5704e8..a0b9af2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala
@@ -22,7 +22,7 @@ import java.sql.Driver
 import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
 
 class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase {
-  test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") {
+  test("setAuthenticationConfig default parser must set authentication all the time") {
     val provider = new MSSQLConnectionProvider()
     val driver = registerDriver(provider.driverClass)
 
@@ -30,7 +30,7 @@ class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase {
       options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql"))
   }
 
-  test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") {
+  test("setAuthenticationConfig custom parser must set authentication all the time") {
     val provider = new MSSQLConnectionProvider() {
       override val parserMethod: String = "IntentionallyNotExistingMethod"
     }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala
index f450662..d8bdf26 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 class MariaDBConnectionProviderSuite extends ConnectionProviderSuiteBase {
-  test("setAuthenticationConfigIfNeeded must set authentication if not set") {
+  test("setAuthenticationConfig must set authentication all the time") {
     val provider = new MariaDBConnectionProvider()
     val driver = registerDriver(provider.driverClass)
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala
index 40e7f11..4aaaf81 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 class OracleConnectionProviderSuite extends ConnectionProviderSuiteBase {
-  test("setAuthenticationConfigIfNeeded must set authentication if not set") {
+  test("setAuthenticationConfig must set authentication all the time") {
     val provider = new OracleConnectionProvider()
     val driver = registerDriver(provider.driverClass)
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala
index ee43a7d..5006bf4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 class PostgresConnectionProviderSuite extends ConnectionProviderSuiteBase {
-  test("setAuthenticationConfigIfNeeded must set authentication if not set") {
+  test("setAuthenticationConfig must set authentication all the time") {
     val provider = new PostgresConnectionProvider()
     val defaultOptions = options("jdbc:postgresql://localhost/postgres")
     val customOptions =


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