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/03/25 09:53:54 UTC

[GitHub] [spark] gaborgsomogyi opened a new pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

gaborgsomogyi opened a new pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019
 
 
   ### What changes were proposed in this pull request?
   When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it.
   
   This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues.
   
   In this PR I've added MariaDB support (other supported databases will come in later PRs).
   
   What this PR contains:
   * Introduced `SecureConnectionProvider` and added basic secure functionalities
   * Added `MariaDBConnectionProvider`
   * Added `MariaDBConnectionProviderSuite`
   * Added `MariaDBKrbIntegrationSuite` docker integration test
   * Added some missing code documentation
   
   ### Why are the changes needed?
   Missing JDBC kerberos support.
   
   ### Does this PR introduce any user-facing change?
   Yes, now user is able to connect to MariaDB using kerberos.
   
   ### How was this patch tested?
   * Additional + existing unit tests
   * Additional + existing integration tests
   * Test on cluster manually
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604262187
 
 
   Hm, I think the tests became considerably flaky lately .. might be best to file a JIRA.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603823144
 
 
   **[Test build #120353 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120353/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403386082
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.Driver
+import javax.security.auth.login.Configuration
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
+
+private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions)
+    extends SecureConnectionProvider(driver, options) {
+  override val appEntry: String = {
+    "Krb5ConnectorContext"
+  }
+
+  override def setAuthenticationConfigIfNeeded(): Unit = {
+    val parent = Configuration.getConfiguration
+    val configEntry = parent.getAppConfigurationEntry(appEntry)
+    /**
+     * Couple of things to mention here:
+     * 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.asScala.exists(o => o._1 == "useKeyTab" && o._2 == "true"))
 
 Review comment:
   `.exists(_ == ("useKeyTab", "true"))`? 
   
   Or even better:
   
   `configEntry.exists(_.getOptions().get("useKeyTab") == "true")`?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603821657
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603916250
 
 
   retest this please

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603908991
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120353/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605861112
 
 
   **[Test build #120581 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120581/testReport)** for PR 28019 at commit [`89f5ac9`](https://github.com/apache/spark/commit/89f5ac93e3a685e08beee4fd79feb846033923e8).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r401647183
 
 

 ##########
 File path: external/docker-integration-tests/pom.xml
 ##########
 @@ -121,8 +121,8 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>mysql</groupId>
-      <artifactId>mysql-connector-java</artifactId>
+      <groupId>org.mariadb.jdbc</groupId>
 
 Review comment:
   Changing from `mysql` to `mariadb` is needed because of this: https://stackoverflow.com/questions/52718788/how-to-read-data-from-mariadb-using-spark-java
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] vanzin closed pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
vanzin closed pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399993325
 
 

 ##########
 File path: external/docker-integration-tests/pom.xml
 ##########
 @@ -121,13 +121,13 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>mysql</groupId>
-      <artifactId>mysql-connector-java</artifactId>
+      <groupId>org.postgresql</groupId>
+      <artifactId>postgresql</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.postgresql</groupId>
-      <artifactId>postgresql</artifactId>
+      <groupId>org.mariadb.jdbc</groupId>
 
 Review comment:
   Yeah, it's not essential so reverted.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603918024
 
 
   **[Test build #120363 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120363/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605946349
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403384639
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
 ##########
 @@ -118,9 +128,11 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu
         .networkDisabled(false)
         .env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava)
         .exposedPorts(s"${db.jdbcPort}/tcp")
+      if(db.getEntryPoint.isDefined) {
 
 Review comment:
   space after if

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604360675
 
 
   cc @HeartSaVioR 

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607204249
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r400013802
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
 ##########
 @@ -53,11 +53,21 @@ abstract class DatabaseOnDocker {
    */
   val jdbcPort: Int
 
+  /**
+   * Optional database name to connect to (not all database drivers need this).
+   */
+  var dbName: Option[String]
+
   /**
    * Return a JDBC URL that connects to the database running at the given IP address and port.
    */
   def getJdbcUrl(ip: String, port: Int): String
 
+  /**
+   * Optional entry point when container starts
+   */
+  def getEntryPoint: Option[String]
 
 Review comment:
   Added.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607203242
 
 
   **[Test build #120670 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120670/testReport)** for PR 28019 at commit [`2bb6426`](https://github.com/apache/spark/commit/2bb6426960227934d2932df5f98d9ea59915af94).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269515
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25115/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609652498
 
 
   **[Test build #120865 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120865/testReport)** for PR 28019 at commit [`2740a50`](https://github.com/apache/spark/commit/2740a507152b5a90fa44400ea9f0cfbcb0ddfcd5).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603826096
 
 
   Filed https://issues.apache.org/jira/browse/SPARK-31247

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603916114
 
 
   Seems unrelated.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605945254
 
 
   **[Test build #120581 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120581/testReport)** for PR 28019 at commit [`89f5ac9`](https://github.com/apache/spark/commit/89f5ac93e3a685e08beee4fd79feb846033923e8).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604260465
 
 
   retest this please

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604265581
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120404/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403912628
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
 ##########
 @@ -41,6 +41,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite {
     override val jdbcPort: Int = 50000
     override def getJdbcUrl(ip: String, port: Int): String =
       s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore
+    override def getEntryPoint: Option[String] = None
 
 Review comment:
   FYI, I've applied the same on `getStartupProcessName`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603908991
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120353/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-606566922
 
 
   cc @vanzin @dongjoon-hyun 

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604265495
 
 
   **[Test build #120404 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120404/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class JDBCConfiguration(`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-611691437
 
 
   @vanzin many thanks for taking care!

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609741721
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604056360
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120363/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603747474
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603821657
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609741732
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120865/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605946353
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120581/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604056348
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604056348
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604265577
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604055592
 
 
   **[Test build #120363 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120363/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class JDBCConfiguration(`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603908977
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604347571
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603821664
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120336/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269509
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604267997
 
 
   retest this please

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604265581
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120404/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269138
 
 
   **[Test build #120407 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120407/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603935012
 
 
   Filed https://issues.apache.org/jira/browse/SPARK-31252

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609740751
 
 
   **[Test build #120865 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120865/testReport)** for PR 28019 at commit [`2740a50`](https://github.com/apache/spark/commit/2740a507152b5a90fa44400ea9f0cfbcb0ddfcd5).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603750866
 
 
   **[Test build #120336 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120336/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604347571
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607118214
 
 
   **[Test build #120670 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120670/testReport)** for PR 28019 at commit [`2bb6426`](https://github.com/apache/spark/commit/2bb6426960227934d2932df5f98d9ea59915af94).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607118214
 
 
   **[Test build #120670 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120670/testReport)** for PR 28019 at commit [`2bb6426`](https://github.com/apache/spark/commit/2bb6426960227934d2932df5f98d9ea59915af94).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603908439
 
 
   **[Test build #120353 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120353/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class JDBCConfiguration(`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399788678
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala
 ##########
 @@ -0,0 +1,75 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.{Connection, Driver}
+import javax.security.auth.login.{AppConfigurationEntry, Configuration}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
+import org.apache.spark.util.SecurityUtils
+
+private[jdbc] abstract class SecureConnectionProvider(driver: Driver, options: JDBCOptions)
 
 Review comment:
   FYI to further reviewers: methods in SecureConnectionProvider (both class and object) are moved from PostgresConnectionProvider.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403911539
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.Driver
+import javax.security.auth.login.Configuration
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
+
+private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions)
+    extends SecureConnectionProvider(driver, options) {
+  override val appEntry: String = {
+    "Krb5ConnectorContext"
+  }
+
+  override def setAuthenticationConfigIfNeeded(): Unit = {
+    val parent = Configuration.getConfiguration
+    val configEntry = parent.getAppConfigurationEntry(appEntry)
+    /**
+     * Couple of things to mention here:
+     * 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.asScala.exists(o => o._1 == "useKeyTab" && o._2 == "true"))
 
 Review comment:
   Fixed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607117991
 
 
   While I'm implementing DB2 kerberos part I've realised that creating new database is not essential for kerberos testing so I've made this simplification in the last commit. Worth to mention re-executed all the docker tests again and all passed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
vanzin commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403384569
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
 ##########
 @@ -41,6 +41,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite {
     override val jdbcPort: Int = 50000
     override def getJdbcUrl(ip: String, port: Int): String =
       s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore
+    override def getEntryPoint: Option[String] = None
 
 Review comment:
   Why not have this in the base class and only override where needed?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269515
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25115/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603750866
 
 
   **[Test build #120336 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120336/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609741721
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607204258
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120670/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607204249
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609652498
 
 
   **[Test build #120865 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120865/testReport)** for PR 28019 at commit [`2740a50`](https://github.com/apache/spark/commit/2740a507152b5a90fa44400ea9f0cfbcb0ddfcd5).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403884556
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala
 ##########
 @@ -41,6 +41,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite {
     override val jdbcPort: Int = 50000
     override def getJdbcUrl(ip: String, port: Int): String =
       s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore
+    override def getEntryPoint: Option[String] = None
 
 Review comment:
   Good simplification, was thinking about this but not yet done. Fixed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605861112
 
 
   **[Test build #120581 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120581/testReport)** for PR 28019 at commit [`89f5ac9`](https://github.com/apache/spark/commit/89f5ac93e3a685e08beee4fd79feb846033923e8).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604360562
 
 
   Filed https://issues.apache.org/jira/browse/SPARK-31266 and https://issues.apache.org/jira/browse/SPARK-31267

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605946353
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120581/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399992584
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.Driver
+import javax.security.auth.login.Configuration
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
+
+private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions)
+    extends SecureConnectionProvider(driver, options) {
+  override val appEntry: String = {
+    "Krb5ConnectorContext"
+  }
+
+  override def setAuthenticationConfigIfNeeded(): Unit = {
+    val parent = Configuration.getConfiguration
+    val configEntry = parent.getAppConfigurationEntry(appEntry)
+    /**
+     * Couple of things to mention here:
+     * 1. MariaDB doesn't support JAAS application name configuration
+     * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" not set
 
 Review comment:
   Fixed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] vanzin commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
vanzin commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-611617373
 
 
   Loks good, merging to master.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604347579
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120407/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604261353
 
 
   **[Test build #120404 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120404/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-609741732
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120865/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399789195
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala
 ##########
 @@ -0,0 +1,69 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.{Driver, DriverManager}
+import javax.security.auth.login.Configuration
+
+import scala.collection.JavaConverters._
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions}
+
+abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndAfterEach {
 
 Review comment:
   FYI to further reviewers: almost everything in ConnectionProviderSuiteBase is moved from PostgreConnectionProviderSuite.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269138
 
 
   **[Test build #120407 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120407/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon edited a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604262187
 
 
   Hm, I think the tests became considerably flaky lately .. yes, might be best to file a JIRA for now ...

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603918024
 
 
   **[Test build #120363 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120363/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604056360
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120363/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399787462
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala
 ##########
 @@ -91,4 +98,66 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite
     logInfo(s"Created executable resource file: ${newEntry.getAbsolutePath}")
     newEntry
   }
+
+  override def dataPreparation(conn: Connection): Unit = {
 
 Review comment:
   FYI to further reviewers: this, and below tests are moved from `PostgreKrbIntegrationSuite`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603908977
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604269509
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604347579
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120407/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-607204258
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120670/
   Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603822376
 
 
   retest this please

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399785210
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
 ##########
 @@ -53,11 +53,21 @@ abstract class DatabaseOnDocker {
    */
   val jdbcPort: Int
 
+  /**
+   * Optional database name to connect to (not all database drivers need this).
+   */
+  var dbName: Option[String]
+
   /**
    * Return a JDBC URL that connects to the database running at the given IP address and port.
    */
   def getJdbcUrl(ip: String, port: Int): String
 
+  /**
+   * Optional entry point when container starts
+   */
+  def getEntryPoint: Option[String]
 
 Review comment:
   Maybe better to elaborate more on `entry point` - which value is expected and how the entry point is supposed to be used.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-605946349
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604260403
 
 
   Seems unrelated.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603820865
 
 
   **[Test build #120336 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120336/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class JDBCConfiguration(`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604261353
 
 
   **[Test build #120404 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120404/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r397730296
 
 

 ##########
 File path: external/docker-integration-tests/pom.xml
 ##########
 @@ -121,13 +121,13 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>mysql</groupId>
-      <artifactId>mysql-connector-java</artifactId>
+      <groupId>org.postgresql</groupId>
+      <artifactId>postgresql</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.postgresql</groupId>
-      <artifactId>postgresql</artifactId>
+      <groupId>org.mariadb.jdbc</groupId>
 
 Review comment:
   Changing from `mysql` to `mariadb` is needed because of this: https://stackoverflow.com/questions/52718788/how-to-read-data-from-mariadb-using-spark-java
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603822330
 
 
   Seems unrelated.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r403890908
 
 

 ##########
 File path: external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
 ##########
 @@ -118,9 +128,11 @@ abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventu
         .networkDisabled(false)
         .env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava)
         .exposedPorts(s"${db.jdbcPort}/tcp")
+      if(db.getEntryPoint.isDefined) {
 
 Review comment:
   Fixed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603821664
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120336/
   Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604265577
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603747474
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-604346466
 
 
   **[Test build #120407 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120407/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  class JDBCConfiguration(`

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#issuecomment-603823144
 
 
   **[Test build #120353 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120353/testReport)** for PR 28019 at commit [`ff04926`](https://github.com/apache/spark/commit/ff04926f2e105c46ba35d57ba075b872791b0b84).

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399788206
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * 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.execution.datasources.jdbc.connection
+
+import java.sql.Driver
+import javax.security.auth.login.Configuration
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
+
+private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptions)
+    extends SecureConnectionProvider(driver, options) {
+  override val appEntry: String = {
+    "Krb5ConnectorContext"
+  }
+
+  override def setAuthenticationConfigIfNeeded(): Unit = {
+    val parent = Configuration.getConfiguration
+    val configEntry = parent.getAppConfigurationEntry(appEntry)
+    /**
+     * Couple of things to mention here:
+     * 1. MariaDB doesn't support JAAS application name configuration
+     * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" not set
 
 Review comment:
   nit: not set -> is not set

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #28019: [SPARK-31021][SQL] Support MariaDB Kerberos login in JDBC connector
URL: https://github.com/apache/spark/pull/28019#discussion_r399199949
 
 

 ##########
 File path: external/docker-integration-tests/pom.xml
 ##########
 @@ -121,13 +121,13 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>mysql</groupId>
-      <artifactId>mysql-connector-java</artifactId>
+      <groupId>org.postgresql</groupId>
+      <artifactId>postgresql</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.postgresql</groupId>
-      <artifactId>postgresql</artifactId>
+      <groupId>org.mariadb.jdbc</groupId>
 
 Review comment:
   Let's revert the order as this unnecessarily adds the change on postgresql.

----------------------------------------------------------------
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


With regards,
Apache Git Services

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