JDBC To Other Databases
- Data Source Option
- Data Type Mapping    - Mapping Spark SQL Data Types from MySQL
- Mapping Spark SQL Data Types to MySQL
- Mapping Spark SQL Data Types from PostgreSQL
- Mapping Spark SQL Data Types to PostgreSQL
- Mapping Spark SQL Data Types from Oracle
- Mapping Spark SQL Data Types to Oracle
- Mapping Spark SQL Data Types from Microsoft SQL Server
- Mapping Spark SQL Data Types to Microsoft SQL Server
- Mapping Spark SQL Data Types from DB2
- Mapping Spark SQL Data Types to DB2
- Mapping Spark SQL Data Types from Teradata
- Mapping Spark SQL Data Types to Teradata
 
Spark SQL also includes a data source that can read data from other databases using JDBC. This functionality should be preferred over using JdbcRDD. This is because the results are returned as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. The JDBC data source is also easier to use from Java or Python as it does not require the user to provide a ClassTag. (Note that this is different than the Spark SQL JDBC server, which allows other applications to run queries using Spark SQL).
To get started you will need to include the JDBC driver for your particular database on the spark classpath. For example, to connect to postgres from the Spark Shell you would run the following command:
./bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jarData Source Option
Spark supports the following case-insensitive options for JDBC. The Data source options of JDBC can be set via:
- the .option/.optionsmethods of- DataFrameReader
- DataFrameWriter
 
- OPTIONSclause at CREATE TABLE USING DATA_SOURCE
For connection properties, users can specify the JDBC connection properties in the data source options.
user and password are normally provided as connection properties for
logging into the data sources.
| Property Name | Default | Meaning | Scope | 
|---|---|---|---|
| url | (none) | The JDBC URL of the form jdbc:subprotocol:subnameto connect to. The source-specific connection properties may be specified in the URL. e.g.,jdbc:postgresql://localhost/test?user=fred&password=secret | read/write | 
| dbtable | (none) | The JDBC table that should be read from or written into. Note that when using it in the read
      path anything that is valid in a FROMclause of a SQL query can be used.
      For example, instead of a full table you could also use a subquery in parentheses. It is not
      allowed to specifydbtableandqueryoptions at the same time. | read/write | 
| query | (none) | A query that will be used to read data into Spark. The specified query will be parenthesized and used
      as a subquery in the FROMclause. Spark will also assign an alias to the subquery clause.
      As an example, spark will issue a query of the following form to the JDBC Source. SELECT <columns> FROM (<user_specified_query>) spark_gen_aliasBelow are a couple of restrictions while using this option. 
 | read | 
| prepareQuery | (none) | A prefix that will form the final query together with query.
      As the specifiedquerywill be parenthesized as a subquery in theFROMclause and some databases do not
      support all clauses in subqueries, theprepareQueryproperty offers a way to run such complex queries.
      As an example, spark will issue a query of the following form to the JDBC Source.<prepareQuery> SELECT <columns> FROM (<user_specified_query>) spark_gen_aliasBelow are a couple of examples. 
 | read/write | 
| driver | (none) | The class name of the JDBC driver to use to connect to this URL. | read/write | 
| partitionColumn, lowerBound, upperBound | (none) | These options must all be specified if any of them is specified. In addition, numPartitionsmust be specified. They describe how to partition the table when
      reading in parallel from multiple workers.partitionColumnmust be a numeric, date, or timestamp column from the table in question.
      Notice thatlowerBoundandupperBoundare just used to decide the
      partition stride, not for filtering the rows in table. So all rows in the table will be
      partitioned and returned. This option applies only to reading.Example: 
         spark.read.format("jdbc") | read | 
| numPartitions | (none) | The maximum number of partitions that can be used for parallelism in table reading and
      writing. This also determines the maximum number of concurrent JDBC connections.
      If the number of partitions to write exceeds this limit, we decrease it to this limit by
      calling coalesce(numPartitions)before writing. | read/write | 
| queryTimeout | 0 | The number of seconds the driver will wait for a Statement object to execute to the given
      number of seconds. Zero means there is no limit. In the write path, this option depends on
      how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver
      checks the timeout of each query instead of an entire JDBC batch. | read/write | 
| fetchsize | 0 | The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (e.g. Oracle with 10 rows). | read | 
| batchsize | 1000 | The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. | write | 
| isolationLevel | READ_UNCOMMITTED | The transaction isolation level, which applies to current connection. It can be one of NONE,READ_COMMITTED,READ_UNCOMMITTED,REPEATABLE_READ, orSERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default ofREAD_UNCOMMITTED. Please refer the documentation injava.sql.Connection. | write | 
| sessionInitStatement | (none) | After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") | read | 
| truncate | false | This is a JDBC writer related option. When SaveMode.Overwriteis enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. In case of failures, users should turn offtruncateoption to useDROP TABLEagain. Also, due to the different behavior ofTRUNCATE TABLEamong DBMSes, it's not always safe to use this. MySQLDialect, DB2Dialect, MsSqlServerDialect, DerbyDialect, and OracleDialect supports this while PostgresDialect and default JDBCDialect doesn't. For unknown and unsupported JDBCDialect, the user optiontruncateis ignored. | write | 
| cascadeTruncate | the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncatein each JDBCDialect | This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE(in the case of PostgreSQL aTRUNCATE TABLE ONLY t CASCADEis executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. | write | 
| createTableOptions |  | This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). | write | 
| createTableColumnTypes | (none) | The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. | write | 
| customSchema | (none) | The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example,"id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. | read | 
| pushDownPredicate | true | The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. | read | 
| pushDownAggregate | true | The option to enable or disable aggregate push-down in V2 JDBC data source. The default value is true, in which case Spark will push down aggregates to the JDBC data source. Otherwise, if sets to false, aggregates will not be pushed down to the JDBC data source. Aggregate push-down is usually turned off when the aggregate is performed faster by Spark than by the JDBC data source. Please note that aggregates can be pushed down if and only if all the aggregate functions and the related filters can be pushed down. If numPartitionsequals to 1 or the group by key is the same aspartitionColumn, Spark will push down aggregate to data source completely and not apply a final aggregate over the data source output. Otherwise, Spark will apply a final aggregate over the data source output. | read | 
| pushDownLimit | true | The option to enable or disable LIMIT push-down into V2 JDBC data source. The LIMIT push-down also includes LIMIT + SORT , a.k.a. the Top N operator. The default value is true, in which case Spark push down LIMIT or LIMIT with SORT to the JDBC data source. Otherwise, if sets to false, LIMIT or LIMIT with SORT is not pushed down to the JDBC data source. If numPartitionsis greater than 1, Spark still applies LIMIT or LIMIT with SORT on the result from data source even if LIMIT or LIMIT with SORT is pushed down. Otherwise, if LIMIT or LIMIT with SORT is pushed down andnumPartitionsequals to 1, Spark will not apply LIMIT or LIMIT with SORT on the result from data source. | read | 
| pushDownOffset | true | The option to enable or disable OFFSET push-down into V2 JDBC data source. The default value is true, in which case Spark will push down OFFSET to the JDBC data source. Otherwise, if sets to false, Spark will not try to push down OFFSET to the JDBC data source. If pushDownOffsetis true andnumPartitionsis equal to 1, OFFSET will be pushed down to the JDBC data source. Otherwise, OFFSET will not be pushed down and Spark still applies OFFSET on the result from data source. | read | 
| pushDownTableSample | true | The option to enable or disable TABLESAMPLE push-down into V2 JDBC data source. The default value is true, in which case Spark push down TABLESAMPLE to the JDBC data source. Otherwise, if value sets to false, TABLESAMPLE is not pushed down to the JDBC data source. | read | 
| keytab | (none) | Location of the kerberos keytab file (which must be pre-uploaded to all nodes either by --filesoption of spark-submit or manually) for the JDBC client. When path information found then Spark considers the keytab distributed manually, otherwise--filesassumed. If bothkeytabandprincipalare defined then Spark tries to do kerberos authentication. | read/write | 
| principal | (none) | Specifies kerberos principal name for the JDBC client. If both keytabandprincipalare defined then Spark tries to do kerberos authentication. | read/write | 
| refreshKrb5Config | false | This option controls whether the kerberos configuration is to be refreshed or not for the JDBC client before
      establishing a new connection. Set to true if you want to refresh the configuration, otherwise set to false.
      The default value is false. Note that if you set this option to true and try to establish multiple connections,
      a race condition can occur. One possible situation would be like as follows. 
 | read/write | 
| connectionProvider | (none) | The name of the JDBC connection provider to use to connect to this URL, e.g. db2,mssql.
      Must be one of the providers loaded with the JDBC data source. Used to disambiguate when more than one provider can handle
      the specified driver and options. The selected provider must not be disabled byspark.sql.sources.disabledJdbcConnProviderList. | read/write | 
| preferTimestampNTZ | false | When the option is set to true, TIMESTAMP WITHOUT TIME ZONE type is inferred as Spark's TimestampNTZ type.
      Otherwise, it is interpreted as Spark's Timestamp type(equivalent to TIMESTAMP WITH LOCAL TIME ZONE).
      This setting specifically affects only the inference of TIMESTAMP WITHOUT TIME ZONE data type. Both TIMESTAMP WITH LOCAL TIME ZONE and TIMESTAMP WITH TIME ZONE data types are consistently interpreted as Spark's Timestamp type regardless of this setting. | read | 
| hint | (none) | This option is used to specify the hint for reading. The supported hint format is a variant of C-style comments: it needs to start with `/*+ ` and end with ` */`. Currently, this option is only supported in MySQLDialect, OracleDialect and DatabricksDialect. | read | 
Note that kerberos authentication with keytab is not always supported by the JDBC driver.
Before using keytab and principal configuration options, please make sure the following requirements are met:
- The included JDBC driver version supports kerberos authentication with keytab.
- There is a built-in connection provider which supports the used database.
There is a built-in connection providers for the following databases:
- DB2
- MariaDB
- MS Sql
- Oracle
- PostgreSQL
If the requirements are not met, please consider using the JdbcConnectionProvider developer API to handle custom authentication.
# Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods
# Loading data from a JDBC source
jdbcDF = spark.read \
    .format("jdbc") \
    .option("url", "jdbc:postgresql:dbserver") \
    .option("dbtable", "schema.tablename") \
    .option("user", "username") \
    .option("password", "password") \
    .load()
jdbcDF2 = spark.read \
    .jdbc("jdbc:postgresql:dbserver", "schema.tablename",
          properties={"user": "username", "password": "password"})
# Specifying dataframe column data types on read
jdbcDF3 = spark.read \
    .format("jdbc") \
    .option("url", "jdbc:postgresql:dbserver") \
    .option("dbtable", "schema.tablename") \
    .option("user", "username") \
    .option("password", "password") \
    .option("customSchema", "id DECIMAL(38, 0), name STRING") \
    .load()
# Saving data to a JDBC source
jdbcDF.write \
    .format("jdbc") \
    .option("url", "jdbc:postgresql:dbserver") \
    .option("dbtable", "schema.tablename") \
    .option("user", "username") \
    .option("password", "password") \
    .save()
jdbcDF2.write \
    .jdbc("jdbc:postgresql:dbserver", "schema.tablename",
          properties={"user": "username", "password": "password"})
# Specifying create table column data types on write
jdbcDF.write \
    .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)") \
    .jdbc("jdbc:postgresql:dbserver", "schema.tablename",
          properties={"user": "username", "password": "password"})// Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods
// Loading data from a JDBC source
val jdbcDF = spark.read
  .format("jdbc")
  .option("url", "jdbc:postgresql:dbserver")
  .option("dbtable", "schema.tablename")
  .option("user", "username")
  .option("password", "password")
  .load()
val connectionProperties = new Properties()
connectionProperties.put("user", "username")
connectionProperties.put("password", "password")
val jdbcDF2 = spark.read
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties)
// Specifying the custom data types of the read schema
connectionProperties.put("customSchema", "id DECIMAL(38, 0), name STRING")
val jdbcDF3 = spark.read
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties)
// Saving data to a JDBC source
jdbcDF.write
  .format("jdbc")
  .option("url", "jdbc:postgresql:dbserver")
  .option("dbtable", "schema.tablename")
  .option("user", "username")
  .option("password", "password")
  .save()
jdbcDF2.write
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties)
// Specifying create table column data types on write
jdbcDF.write
  .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)")
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties)// Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods
// Loading data from a JDBC source
Dataset<Row> jdbcDF = spark.read()
  .format("jdbc")
  .option("url", "jdbc:postgresql:dbserver")
  .option("dbtable", "schema.tablename")
  .option("user", "username")
  .option("password", "password")
  .load();
Properties connectionProperties = new Properties();
connectionProperties.put("user", "username");
connectionProperties.put("password", "password");
Dataset<Row> jdbcDF2 = spark.read()
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties);
// Saving data to a JDBC source
jdbcDF.write()
  .format("jdbc")
  .option("url", "jdbc:postgresql:dbserver")
  .option("dbtable", "schema.tablename")
  .option("user", "username")
  .option("password", "password")
  .save();
jdbcDF2.write()
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties);
// Specifying create table column data types on write
jdbcDF.write()
  .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)")
  .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties);# Loading data from a JDBC source
df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password")
# Saving data to a JDBC source
write.jdbc(df, "jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password")CREATE TEMPORARY VIEW jdbcTable
USING org.apache.spark.sql.jdbc
OPTIONS (
  url "jdbc:postgresql:dbserver",
  dbtable "schema.tablename",
  user 'username',
  password 'password'
)
INSERT INTO TABLE jdbcTable
SELECT * FROM resultTableData Type Mapping
Mapping Spark SQL Data Types from MySQL
The below table describes the data type conversions from MySQL data types to Spark SQL Data Types, when reading data from a MySQL table using the built-in jdbc data source with the MySQL Connector/J as the activated JDBC Driver. Note that, different JDBC drivers, such as Maria Connector/J, which are also available to connect MySQL, may have different mapping rules.
| MySQL Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| BIT(1) | BooleanType | |
| BIT( >1 ) | BinaryType | (Default) | 
| BIT( >1 ) | LongType | spark.sql.legacy.mysql.bitArrayMapping.enabled=true | 
| TINYINT(1) | BooleanType | |
| TINYINT(1) | ByteType | tinyInt1isBit=false | 
| BOOLEAN | BooleanType | |
| BOOLEAN | ByteType | tinyInt1isBit=false | 
| TINYINT( >1 ) | ByteType | |
| TINYINT( any ) UNSIGNED | ShortType | |
| SMALLINT | ShortType | |
| SMALLINT UNSIGNED | IntegerType | |
| MEDIUMINT [UNSIGNED] | IntegerType | |
| INT | IntegerType | |
| INT UNSIGNED | LongType | |
| BIGINT | LongType | |
| BIGINT UNSIGNED | DecimalType(20,0) | |
| FLOAT | FloatType | |
| FLOAT UNSIGNED | DoubleType | |
| DOUBLE [UNSIGNED] | DoubleType | |
| DECIMAL(p,s) [UNSIGNED] | DecimalType(min(38, p),(min(18,s))) | The column type is bounded to DecimalType(38, 18), if 'p>38', the fraction part will be truncated if exceeded. And if any value of this column have an actual precision greater 38 will fail with NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION error | 
| DATE | DateType | |
| DATETIME | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| DATETIME | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| TIMESTAMP | TimestampType | |
| TIME | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIME | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| YEAR | DateType | yearIsDateType=true | 
| YEAR | IntegerType | yearIsDateType=false | 
| CHAR(n) | CharType(n) | |
| VARCHAR(n) | VarcharType(n) | |
| BINARY(n) | BinaryType | |
| VARBINARY(n) | BinaryType | |
| CHAR(n) BINARY | BinaryType | |
| VARCHAR(n) BINARY | BinaryType | |
| BLOB | BinaryType | |
| TINYBLOB | BinaryType | |
| MEDIUMBLOB | BinaryType | |
| LONGBLOB | BinaryType | |
| TEXT | StringType | |
| TINYTEXT | StringType | |
| MEDIUMTEXT | StringType | |
| LONGTEXT | StringType | |
| JSON | StringType | |
| GEOMETRY | BinaryType | |
| ENUM | CharType(n) | |
| SET | CharType(n) | 
Mapping Spark SQL Data Types to MySQL
The below table describes the data type conversions from Spark SQL Data Types to MySQL data types, when creating, altering, or writing data to a MySQL table using the built-in jdbc data source with the MySQL Connector/J as the activated JDBC Driver.
Note that, different JDBC drivers, such as Maria Connector/J, which are also available to connect MySQL, may have different mapping rules.
| Spark SQL Data Type | MySQL Data Type | Remarks | 
|---|---|---|
| BooleanType | BIT(1) | |
| ByteType | TINYINT | |
| ShortType | SMALLINT | For Spark 3.5 and previous, it maps to INTEGER | 
| IntegerType | INTEGER | |
| LongType | BIGINT | |
| FloatType | FLOAT | |
| DoubleType | DOUBLE PRECISION | |
| DecimalType(p, s) | DECIMAL(p,s) | |
| DateType | DATE | |
| TimestampType | TIMESTAMP | |
| TimestampNTZType | DATETIME | |
| StringType | LONGTEXT | |
| BinaryType | BLOB | |
| CharType(n) | CHAR(n) | |
| VarcharType(n) | VARCHAR(n) | 
The Spark Catalyst data types below are not supported with suitable MYSQL types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType
Mapping Spark SQL Data Types from PostgreSQL
The below table describes the data type conversions from PostgreSQL data types to Spark SQL Data Types, when reading data from a Postgres table using the built-in jdbc data source with the PostgreSQL JDBC Driver as the activated JDBC Driver. Note that, different JDBC drivers, or different versions might result slightly different.
| PostgreSQL Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| boolean | BooleanType | |
| smallint, smallserial | ShortType | |
| integer, serial | IntegerType | |
| bigint, bigserial | LongType | |
| float, float(p), real | FloatType | 1 ≤ p ≤ 24 | 
| float(p) | DoubleType | 25 ≤ p ≤ 53 | 
| double precision | DoubleType | |
| numeric, decimal | DecimalType | 
 | 
| character varying(n), varchar(n) | VarcharType(n) | |
| character(n), char(n), bpchar(n) | CharType(n) | |
| bpchar | StringType | |
| text | StringType | |
| bytea | BinaryType | |
| date | DateType | |
| timestamp [ (p) ] [ without time zone ] | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| timestamp [ (p) ] [ without time zone ] | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| timestamp [ (p) ] with time zone | TimestampType | |
| time [ (p) ] [ without time zone ] | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| time [ (p) ] [ without time zone ] | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| time [ (p) ] with time zone | TimestampType | |
| interval [ fields ] [ (p) ] | StringType | |
| ENUM | StringType | |
| money | StringType | Monetary Types | 
| inet, cidr, macaddr, macaddr8 | StringType | Network Address Types | 
| point, line, lseg, box, path, polygon, circle | StringType | Geometric Types | 
| pg_lsn | StringType | Log Sequence Number | 
| bit, bit(1) | BooleanType | |
| bit( >1 ) | BinaryType | |
| bit varying( any ) | BinaryType | |
| tsvector, tsquery | StringType | Text Search Types | 
| uuid | StringType | Universally Unique Identifier Type | 
| xml | StringType | XML Type | 
| json, jsonb | StringType | JSON Types | 
| array | ArrayType | |
| Composite Types | StringType | Types created by CREATE TYPE syntax. | 
| int4range, int8range, numrange, tsrange, tstzrange, daterange, etc | StringType | Range Types | 
| Domain Types | (Decided by the underlying type) | |
| oid | DecimalType(20, 0) | Object Identifier Types | 
| regxxx | StringType | Object Identifier Types | 
| void | NullType | void is a Postgres pseudo type, other pseudo types have not yet been verified | 
Mapping Spark SQL Data Types to PostgreSQL
The below table describes the data type conversions from Spark SQL Data Types to PostgreSQL data types, when creating, altering, or writing data to a PostgreSQL table using the built-in jdbc data source with the PostgreSQL JDBC Driver as the activated JDBC Driver.
| Spark SQL Data Type | PostgreSQL Data Type | Remarks | ||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
| BooleanType | boolean | |||||||||||||||||||||||||||||||||
| ByteType | smallint | |||||||||||||||||||||||||||||||||
| ShortType | smallint | |||||||||||||||||||||||||||||||||
| IntegerType | integer | |||||||||||||||||||||||||||||||||
| LongType | bigint | |||||||||||||||||||||||||||||||||
| FloatType | float4 | |||||||||||||||||||||||||||||||||
| DoubleType | float8 | |||||||||||||||||||||||||||||||||
| DecimalType(p, s) | numeric(p,s) | |||||||||||||||||||||||||||||||||
| DateType | date | |||||||||||||||||||||||||||||||||
| TimestampType | timestamp with time zone | Before Spark 4.0, it was mapped as timestamp. Please refer to the migration guide for more information | ||||||||||||||||||||||||||||||||
| TimestampNTZType | timestamp | |||||||||||||||||||||||||||||||||
| StringType | text | |||||||||||||||||||||||||||||||||
| BinaryType | bytea | |||||||||||||||||||||||||||||||||
| CharType(n) | CHAR(n) | |||||||||||||||||||||||||||||||||
| VarcharType(n) | VARCHAR(n) | |||||||||||||||||||||||||||||||||
| ArrayType | 
 | If the element type is an ArrayType, it converts to Postgres multidimensional array. For instance, ArrayType(ArrayType(StringType))converts totext[][],ArrayType(ArrayType(ArrayType(LongType)))converts tobigint[][][] | 
The Spark Catalyst data types below are not supported with suitable PostgreSQL types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType - if the element type is not listed above
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType
Mapping Spark SQL Data Types from Oracle
The below table describes the data type conversions from Oracle data types to Spark SQL Data Types, when reading data from an Oracle table using the built-in jdbc data source with the Oracle JDBC as the activated JDBC Driver.
| Oracle Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| BOOLEAN | BooleanType | Introduced since Oracle Release 23c | 
| NUMBER[(p[,s])] | DecimalType(p,s) | 's' can be negative in Oracle. If 's<0' it'll be adjusted to DecimalType(min(p-s, 38), 0); Otherwise, DecimalType(p, s), and if 'p>38', the fraction part will be truncated if exceeded. And if any value of this column have an actual precision greater 38 will fail with NUMERIC_VALUE_OUT_OF_RANGE.WITHOUT_SUGGESTION error | 
| FLOAT[(p)] | DecimalType(38, 10) | |
| BINARY_FLOAT | FloatType | |
| BINARY_DOUBLE | DoubleType | |
| LONG | BinaryType | |
| RAW(size) | BinaryType | |
| LONG RAW | BinaryType | |
| DATE | TimestampType | When oracle.jdbc.mapDateToTimestamp=true, it follows TIMESTAMP's behavior below | 
| DATE | DateType | When oracle.jdbc.mapDateToTimestamp=false, it maps to DateType | 
| TIMESTAMP | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIMESTAMP | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| TIMESTAMP WITH TIME ZONE | TimestampType | |
| TIMESTAMP WITH LOCAL TIME ZONE | TimestampType | |
| INTERVAL YEAR TO MONTH | YearMonthIntervalType | |
| INTERVAL DAY TO SECOND | DayTimeIntervalType | |
| CHAR[(size [BYTE | CHAR])] | CharType(size) | |
| NCHAR[(size)] | StringType | |
| VARCHAR2(size [BYTE | CHAR]) | VarcharType(size) | |
| NVARCHAR2 | StringType | |
| ROWID/UROWID | StringType | |
| CLOB | StringType | |
| NCLOB | StringType | |
| BLOB | BinaryType | |
| BFILE | UNRECOGNIZED_SQL_TYPE error raised | 
Mapping Spark SQL Data Types to Oracle
The below table describes the data type conversions from Spark SQL Data Types to Oracle data types, when creating, altering, or writing data to an Oracle table using the built-in jdbc data source with the Oracle JDBC as the activated JDBC Driver.
| Spark SQL Data Type | Oracle Data Type | Remarks | 
|---|---|---|
| BooleanType | NUMBER(1, 0) | BooleanType maps to NUMBER(1, 0) as BOOLEAN is introduced since Oracle Release 23c | 
| ByteType | NUMBER(3) | |
| ShortType | NUMBER(5) | |
| IntegerType | NUMBER(10) | |
| LongType | NUMBER(19) | |
| FloatType | NUMBER(19, 4) | |
| DoubleType | NUMBER(19, 4) | |
| DecimalType(p, s) | NUMBER(p,s) | |
| DateType | DATE | |
| TimestampType | TIMESTAMP WITH LOCAL TIME ZONE | |
| TimestampNTZType | TIMESTAMP | |
| StringType | VARCHAR2(255) | For historical reason, a string value has maximum 255 characters | 
| BinaryType | BLOB | |
| CharType(n) | CHAR(n) | |
| VarcharType(n) | VARCHAR2(n) | 
The Spark Catalyst data types below are not supported with suitable Oracle types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType
Mapping Spark SQL Data Types from Microsoft SQL Server
The below table describes the data type conversions from Microsoft SQL Server data types to Spark SQL Data Types, when reading data from a Microsoft SQL Server table using the built-in jdbc data source with the mssql-jdbc as the activated JDBC Driver.
| SQL Server Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| bit | BooleanType | |
| tinyint | ShortType | |
| smallint | ShortType | |
| int | IntegerType | |
| bigint | LongType | |
| float(p), real | FloatType | 1 ≤ p ≤ 24 | 
| float[(p)] | DoubleType | 25 ≤ p ≤ 53 | 
| double precision | DoubleType | |
| smallmoney | DecimalType(10, 4) | |
| money | DecimalType(19, 4) | |
| decimal[(p[, s])], numeric[(p[, s])] | DecimalType(p, s) | |
| date | DateType | |
| datetime | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| datetime | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| datetime2 [ (fractional seconds precision) ] | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| datetime2 [ (fractional seconds precision) ] | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| datetimeoffset [ (fractional seconds precision) ] | TimestampType | |
| smalldatetime | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| smalldatetime | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| time [ (fractional second scale) ] | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| time [ (fractional second scale) ] | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| binary [ ( n ) ] | BinaryType | |
| varbinary [ ( n | max ) ] | BinaryType | |
| char [ ( n ) ] | CharType(n) | |
| varchar [ ( n | max ) ] | VarcharType(n) | |
| nchar [ ( n ) ] | StringType | |
| nvarchar [ ( n | max ) ] | StringType | |
| text | StringType | |
| ntext | StringType | |
| image | StringType | |
| geography | BinaryType | |
| geometry | BinaryType | |
| rowversion | BinaryType | |
| sql_variant | UNRECOGNIZED_SQL_TYPE error raised | 
Mapping Spark SQL Data Types to Microsoft SQL Server
The below table describes the data type conversions from Spark SQL Data Types to Microsoft SQL Server data types, when creating, altering, or writing data to a Microsoft SQL Server table using the built-in jdbc data source with the mssql-jdbc as the activated JDBC Driver.
| Spark SQL Data Type | SQL Server Data Type | Remarks | 
|---|---|---|
| BooleanType | bit | |
| ByteType | smallint | Supported since Spark 4.0.0, previous versions throw errors | 
| ShortType | smallint | |
| IntegerType | int | |
| LongType | bigint | |
| FloatType | real | |
| DoubleType | double precision | |
| DecimalType(p, s) | number(p,s) | |
| DateType | date | |
| TimestampType | datetime | |
| TimestampNTZType | datetime | |
| StringType | nvarchar(max) | |
| BinaryType | varbinary(max) | |
| CharType(n) | char(n) | |
| VarcharType(n) | varchar(n) | 
The Spark Catalyst data types below are not supported with suitable SQL Server types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType
Mapping Spark SQL Data Types from DB2
The below table describes the data type conversions from DB2 data types to Spark SQL Data Types, when reading data from a DB2 table using the built-in jdbc data source with the IBM Data Server Driver For JDBC and SQLJ as the activated JDBC Driver.
| DB2 Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| BOOLEAN | BinaryType | |
| SMALLINT | ShortType | |
| INTEGER | IntegerType | |
| BIGINT | LongType | |
| REAL | FloatType | |
| DOUBLE, FLOAT | DoubleType | FLOAT is double precision floating-point in db2 | 
| DECIMAL, NUMERIC, DECFLOAT | DecimalType | |
| DATE | DateType | |
| TIMESTAMP, TIMESTAMP WITHOUT TIME ZONE | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIMESTAMP, TIMESTAMP WITHOUT TIME ZONE | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| TIMESTAMP WITH TIME ZONE | TimestampType | |
| TIME | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIME | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| CHAR(n) | CharType(n) | |
| VARCHAR(n) | VarcharType(n) | |
| CHAR(n) FOR BIT DATA | BinaryType | |
| VARCHAR(n) FOR BIT DATA | BinaryType | |
| BINARY(n) | BinaryType | |
| VARBINARY(n) | BinaryType | |
| CLOB(n) | StringType | |
| DBCLOB(n) | StringType | |
| BLOB(n) | BinaryType | |
| GRAPHIC(n) | StringType | |
| VARGRAPHIC(n) | StringType | |
| XML | StringType | |
| ROWID | StringType | 
Mapping Spark SQL Data Types to DB2
The below table describes the data type conversions from Spark SQL Data Types to DB2 data types, when creating, altering, or writing data to a DB2 table using the built-in jdbc data source with the IBM Data Server Driver For JDBC and SQLJ as the activated JDBC Driver.
| Spark SQL Data Type | DB2 Data Type | Remarks | 
|---|---|---|
| BooleanType | BOOLEAN | |
| ByteType | SMALLINT | |
| ShortType | SMALLINT | |
| IntegerType | INTEGER | |
| LongType | BIGINT | |
| FloatType | REAL | |
| DoubleType | DOUBLE PRECISION | |
| DecimalType(p, s) | DECIMAL(p,s) | The maximum value for 'p' is 31 in DB2, while it is 38 in Spark. It might fail when storing DecimalType(p>=32, s) to DB2 | 
| DateType | DATE | |
| TimestampType | TIMESTAMP | |
| TimestampNTZType | TIMESTAMP | |
| StringType | CLOB | |
| BinaryType | BLOB | |
| CharType(n) | CHAR(n) | The maximum value for 'n' is 255 in DB2, while it is unlimited in Spark. | 
| VarcharType(n) | VARCHAR(n) | The maximum value for 'n' is 255 in DB2, while it is unlimited in Spark. | 
The Spark Catalyst data types below are not supported with suitable DB2 types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType
Mapping Spark SQL Data Types from Teradata
The below table describes the data type conversions from Teradata data types to Spark SQL Data Types, when reading data from a Teradata table using the built-in jdbc data source with the Teradata JDBC Driver as the activated JDBC Driver.
| Teradata Data Type | Spark SQL Data Type | Remarks | 
|---|---|---|
| BYTEINT | ByteType | |
| SMALLINT | ShortType | |
| INTEGER, INT | IntegerType | |
| BIGINT | LongType | |
| REAL, DOUBLE PRECISION, FLOAT | DoubleType | |
| DECIMAL, NUMERIC, NUMBER | DecimalType | |
| DATE | DateType | |
| TIMESTAMP, TIMESTAMP WITH TIME ZONE | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIMESTAMP, TIMESTAMP WITH TIME ZONE | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| TIME, TIME WITH TIME ZONE | TimestampType | (Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ | 
| TIME, TIME WITH TIME ZONE | TimestampNTZType | preferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ | 
| CHARACTER(n), CHAR(n), GRAPHIC(n) | CharType(n) | |
| VARCHAR(n), VARGRAPHIC(n) | VarcharType(n) | |
| BYTE(n), VARBYTE(n) | BinaryType | |
| CLOB | StringType | |
| BLOB | BinaryType | |
| INTERVAL Data Types | - | The INTERVAL data types are unknown yet | 
| Period Data Types, ARRAY, UDT | - | Not Supported | 
Mapping Spark SQL Data Types to Teradata
The below table describes the data type conversions from Spark SQL Data Types to Teradata data types, when creating, altering, or writing data to a Teradata table using the built-in jdbc data source with the Teradata JDBC Driver as the activated JDBC Driver.
| Spark SQL Data Type | Teradata Data Type | Remarks | 
|---|---|---|
| BooleanType | CHAR(1) | |
| ByteType | BYTEINT | |
| ShortType | SMALLINT | |
| IntegerType | INTEGER | |
| LongType | BIGINT | |
| FloatType | REAL | |
| DoubleType | DOUBLE PRECISION | |
| DecimalType(p, s) | DECIMAL(p,s) | |
| DateType | DATE | |
| TimestampType | TIMESTAMP | |
| TimestampNTZType | TIMESTAMP | |
| StringType | VARCHAR(255) | |
| BinaryType | BLOB | |
| CharType(n) | CHAR(n) | |
| VarcharType(n) | VARCHAR(n) | 
The Spark Catalyst data types below are not supported with suitable Teradata types.
- DayTimeIntervalType
- YearMonthIntervalType
- CalendarIntervalType
- ArrayType
- MapType
- StructType
- UserDefinedType
- NullType
- ObjectType
- VariantType