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

org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.saveTable the case senstivity issue

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Resolved
    • 2.0.1
    • 2.2.0
    • SQL
    • None

    Description

      Blindly quoting every field name for inserting is the issue (Line 110-119, https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala).

      /**

      • Returns a PreparedStatement that inserts a row into table via conn.
        */
        def insertStatement(conn: Connection, table: String, rddSchema: StructType, dialect: JdbcDialect)
        : PreparedStatement = { val columns = rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",") val placeholders = rddSchema.fields.map(_ => "?").mkString(",") val sql = s"INSERT INTO $table ($columns) VALUES ($placeholders)" conn.prepareStatement(sql) }

      This code causes the following issue (it does not happen to 1.6.x):

      I have issue with the saveTable method in Spark 2.0/2.0.1. I tried to save a dataset to Oracle database, but the fields must be uppercase to succeed. This is not an expected behavior: If only the table names were quoted, this utility should concern the case sensitivity. The code below throws the exception: Caused by: java.sql.SQLSyntaxErrorException: ORA-00904: "DATETIME_gmt": invalid identifier.

      String detailSQL ="select CAST('2016-09-25 17:00:00' AS TIMESTAMP) DATETIME_gmt, '1' NODEB";
      hc.sql("set spark.sql.caseSensitive=false");
      Dataset<Row> ds = hc.sql(detailSQL);
      ds.show();
      org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.saveTable(ds, url, detailTable, p);

      Attachments

        Issue Links

          Activity

            People

              dongjoon Dongjoon Hyun
              zwu.net@gmail.com Paul Wu
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: