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.jar
Data Source Option
Spark supports the following case-insensitive options for JDBC. The Data source options of JDBC can be set via:
.option
/.options
methods of
DataFrameReader
DataFrameWriter
OPTIONS
clause at CREATE TABLE USING DATA_SOURCEFor 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.
url
(none) The JDBC URL of the form jdbc:subprotocol:subname
to 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 FROM
clause 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 specify dbtable
and query
options 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 FROM
clause. 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_alias
Below are a couple of restrictions while using this option.
dbtable
and query
options at the same time.query
and partitionColumn
options at the same time. When specifying partitionColumn
option is required, the subquery can be specified using dbtable
option instead and partition columns can be qualified using the subquery alias provided as part of dbtable
. spark.read.format("jdbc")
.option("url", jdbcUrl)
.option("query", "select c1, c2 from t1")
.load()
prepareQuery
(none) A prefix that will form the final query together with query
. As the specified query
will be parenthesized as a subquery in the FROM
clause and some databases do not support all clauses in subqueries, the prepareQuery
property 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_alias
Below are a couple of examples.
WITH
clauses in subqueries but it is possible to split such a query to prepareQuery
and query
: spark.read.format("jdbc")
.option("url", jdbcUrl)
.option("prepareQuery", "WITH t AS (SELECT x, y FROM tbl)")
.option("query", "SELECT * FROM t WHERE x > 10")
.load()
prepareQuery
and query
: spark.read.format("jdbc")
.option("url", jdbcUrl)
.option("prepareQuery", "(SELECT * INTO #TempTable FROM (SELECT * FROM tbl) t)")
.option("query", "SELECT * FROM #TempTable")
.load()
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, numPartitions
must be specified. They describe how to partition the table when reading in parallel from multiple workers. partitionColumn
must be a numeric, date, or timestamp column from the table in question. Notice that lowerBound
and upperBound
are 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.
spark.read.format("jdbc")
.option("url", jdbcUrl)
.option("dbtable", "(select c1, c2 from t1) as subq")
.option("partitionColumn", "c1")
.option("lowerBound", "1")
.option("upperBound", "100")
.option("numPartitions", "3")
.load()
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
, or SERIALIZABLE
, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED
. Please refer the documentation in java.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.Overwrite
is 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 off truncate
option to use DROP TABLE
again. Also, due to the different behavior of TRUNCATE TABLE
among 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 option truncate
is ignored. write cascadeTruncate
the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate
in 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 a TRUNCATE TABLE ONLY t CASCADE
is 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 numPartitions
equals to 1 or the group by key is the same as partitionColumn
, 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 numPartitions
is 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 and numPartitions
equals 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 pushDownOffset
is true and numPartitions
is 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 --files
option of spark-submit or manually) for the JDBC client. When path information found then Spark considers the keytab distributed manually, otherwise --files
assumed. If both keytab
and principal
are defined then Spark tries to do kerberos authentication. read/write principal
(none) Specifies kerberos principal name for the JDBC client. If both keytab
and principal
are 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.
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 by spark.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:
There is a built-in connection providers for the following databases:
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"})
Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.
// 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)
Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.
// 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);
Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.
# 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")
Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.
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 resultTable
Data 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 MySQLThe 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.
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 DecimalTypeThe 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 Element type PG Array BooleanType boolean[] ByteType smallint[] ShortType smallint[] IntegerType integer[] LongType bigint[] FloatType float4[] DoubleType float8[] DecimalType(p, s) numeric(p,s)[] DateType date[] TimestampType timestamp[] TimestampNTZType timestamp[] StringType text[] BinaryType bytea[] CharType(n) char(n)[] VarcharType(n) varchar(n)[] If the element type is an ArrayType, it converts to Postgres multidimensional array.ArrayType(ArrayType(StringType))
converts to text[][]
,
ArrayType(ArrayType(ArrayType(LongType)))
converts to bigint[][][]
The Spark Catalyst data types below are not supported with suitable PostgreSQL types.
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 OracleThe 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.
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 ServerThe 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.
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 DB2The 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.
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 TeradataThe 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.
RetroSearch is an open source project built by @garambo | Open a GitHub Issue
Search and Browse the WWW like it's 1997 | Search results from DuckDuckGo
HTML:
3.2
| Encoding:
UTF-8
| Version:
0.7.4