You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/07 12:36:07 UTC

[GitHub] [spark] maropu commented on a change in pull request #29024: [SPARK-32001][SQL]Create JDBC authentication provider developer API

maropu commented on a change in pull request #29024:
URL: https://github.com/apache/spark/pull/29024#discussion_r500975741



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala
##########
@@ -18,60 +18,45 @@
 package org.apache.spark.sql.execution.datasources.jdbc.connection
 
 import java.sql.{Connection, Driver}
-import java.util.Properties
+import java.util.ServiceLoader
 
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
-
-/**
- * Connection provider which opens connection toward various databases (database specific instance
- * needed). If kerberos authentication required then it's the provider's responsibility to set all
- * the parameters.
- */
-private[jdbc] trait ConnectionProvider {
-  /**
-   * Additional properties for data connection (Data source property takes precedence).
-   */
-  def getAdditionalProperties(): Properties = new Properties()
+import scala.collection.mutable
 
-  /**
-   * Opens connection toward the database.
-   */
-  def getConnection(): Connection
-}
+import org.apache.spark.internal.Logging
+import org.apache.spark.security.SecurityConfigurationLock
+import org.apache.spark.sql.jdbc.JdbcConnectionProvider
+import org.apache.spark.util.Utils
 
 private[jdbc] object ConnectionProvider extends Logging {
-  def create(driver: Driver, options: JDBCOptions): ConnectionProvider = {
-    if (options.keytab == null || options.principal == null) {
-      logDebug("No authentication configuration found, using basic connection provider")
-      new BasicConnectionProvider(driver, options)
-    } else {
-      logDebug("Authentication configuration found, using database specific connection provider")
-      options.driverClass match {
-        case PostgresConnectionProvider.driverClass =>
-          logDebug("Postgres connection provider found")
-          new PostgresConnectionProvider(driver, options)
-
-        case MariaDBConnectionProvider.driverClass =>
-          logDebug("MariaDB connection provider found")
-          new MariaDBConnectionProvider(driver, options)
-
-        case DB2ConnectionProvider.driverClass =>
-          logDebug("DB2 connection provider found")
-          new DB2ConnectionProvider(driver, options)
-
-        case MSSQLConnectionProvider.driverClass =>
-          logDebug("MS SQL connection provider found")
-          new MSSQLConnectionProvider(driver, options)
-
-        case OracleConnectionProvider.driverClass =>
-          logDebug("Oracle connection provider found")
-          new OracleConnectionProvider(driver, options)
-
-        case _ =>
-          throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " +
-            "Kerberos authentication")
+  private val providers = loadProviders()
+
+  def loadProviders(): Seq[JdbcConnectionProvider] = {
+    val loader = ServiceLoader.load(classOf[JdbcConnectionProvider],
+      Utils.getContextOrSparkClassLoader)
+    val providers = mutable.ArrayBuffer[JdbcConnectionProvider]()
+
+    val iterator = loader.iterator
+    while (iterator.hasNext) {
+      try {
+        val provider = iterator.next
+        logDebug(s"Loaded built in provider: $provider")
+        providers += provider
+      } catch {
+        case t: Throwable =>
+          logError(s"Failed to load built in provider.", t)

Review comment:
       Is it okay to ignore the error case where it fails to load *builtin* providers? `DataSource` throws an exception if it fails to load builtin datasources:
   https://github.com/apache/spark/blob/94d648dff5f24b4dea3873fd8e6609b1a099d0a2/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L694-L702




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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