Skip to content
Snippets Groups Projects
Commit 502476e4 authored by CK50's avatar CK50 Committed by Sean Owen
Browse files

[SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT syntax

In the past Spark JDBC write only worked with technologies which support the following INSERT statement syntax (JdbcUtils.scala: insertStatement()):

INSERT INTO $table VALUES ( ?, ?, ..., ? )

But some technologies require a list of column names:

INSERT INTO $table ( $colNameList ) VALUES ( ?, ?, ..., ? )

This was blocking the use of e.g. the Progress JDBC Driver for Cassandra.

Another limitation is that syntax 1 relies no the dataframe field ordering match that of the target table. This works fine, as long as the target table has been created by writer.jdbc().

If the target table contains more columns (not created by writer.jdbc()), then the insert fails due mismatch of number of columns or their data types.

This PR switches to the recommended second INSERT syntax. Column names are taken from datafram field names.

Author: CK50 <christian.kurz@oracle.com>

Closes #10380 from CK50/master-SPARK-12010-2.
parent 39204661
No related branches found
No related tags found
No related merge requests found
......@@ -63,14 +63,10 @@ object JdbcUtils extends Logging {
* Returns a PreparedStatement that inserts a row into table via conn.
*/
def insertStatement(conn: Connection, table: String, rddSchema: StructType): PreparedStatement = {
val sql = new StringBuilder(s"INSERT INTO $table VALUES (")
var fieldsLeft = rddSchema.fields.length
while (fieldsLeft > 0) {
sql.append("?")
if (fieldsLeft > 1) sql.append(", ") else sql.append(")")
fieldsLeft = fieldsLeft - 1
}
conn.prepareStatement(sql.toString())
val columns = rddSchema.fields.map(_.name).mkString(",")
val placeholders = rddSchema.fields.map(_ => "?").mkString(",")
val sql = s"INSERT INTO $table ($columns) VALUES ($placeholders)"
conn.prepareStatement(sql)
}
/**
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment