Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-14204

[SQL] Failure to register URL-derived JDBC driver on executors in cluster mode

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.1
    • 1.6.2, 2.0.1, 2.1.0
    • SQL

    Description

      DataFrameReader JDBC methods throw an IllegalStateException when:
      1. the JDBC driver is contained in a user-provided jar, and
      2. the user does not specify which driver to use, but rather allows spark to determine the driver from the JDBC URL.

      This broke some of our database ETL jobs at @premisedata when we upgraded from 1.6.0 to 1.6.1.

      I have tracked the problem down to a regression introduced in the fix for SPARK-12579: https://github.com/apache/spark/commit/7f37c1e45d52b7823d566349e2be21366d73651f#diff-391379a5ec51082e2ae1209db15c02b3R53

      The issue is that DriverRegistry.register is not called on the executors for a JDBC driver that is derived from the JDBC path.

      The problem can be demonstrated within spark-shell, provided you're in cluster mode and you've deployed a JDBC driver (e.g. postgresql.Driver) via the --jars argument:

      import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.createConnectionFactory
      val factory = createConnectionFactory("jdbc:postgresql://whatever.you.want/database?user=user&password=password", new java.util.Properties)
      sc.parallelize(1 to 100).foreach { _ => factory() } // throws exception
      

      A sufficient fix is to apply DriverRegistry.register to the `driverClass` variable, rather than to `userSpecifiedDriverClass`, at the code link provided above. I will submit a PR for this shortly.

      In the meantime, a temporary workaround is to manually specify the JDBC driver class in the Properties object passed to DataFrameReader.jdbc, or in the options used in other entry points, which will force the executors to register the class properly.

      Attachments

        Issue Links

          Activity

            People

              mchalek Kevin McHale
              mchalek Kevin McHale
              Votes:
              3 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: