Skip to content
6 changes: 6 additions & 0 deletions external/docker-integration-tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -150,5 +150,11 @@
<version>10.5.0.5</version>
<type>jar</type>
</dependency>
<dependency>
<groupId>com.microsoft.sqlserver</groupId>
<artifactId>mssql-jdbc</artifactId>
<version>7.2.1.jre8</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,205 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.jdbc

import java.math.BigDecimal
import java.sql.{Connection, Date, Timestamp}
import java.util.Properties

import org.apache.spark.tags.DockerTest

@DockerTest
class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite {
override val db = new DatabaseOnDocker {
override val imageName = "mcr.microsoft.com/mssql/server:2017-GA-ubuntu"
override val env = Map(
"SA_PASSWORD" -> "Sapass123",
"ACCEPT_EULA" -> "Y"
)
override val usesIpc = false
override val jdbcPort: Int = 1433

override def getJdbcUrl(ip: String, port: Int): String =
s"jdbc:sqlserver://$ip:$port;user=sa;password=Sapass123;"

override def getStartupProcessName: Option[String] = None
}

override def dataPreparation(conn: Connection): Unit = {
conn.prepareStatement("CREATE TABLE tbl (x INT, y VARCHAR (50))").executeUpdate()
conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate()
conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate()

conn.prepareStatement(
"""
|CREATE TABLE numbers (
|a BIT,
|b TINYINT, c SMALLINT, d INT, e BIGINT,
|f FLOAT, f1 FLOAT(24),
|g REAL,
|h DECIMAL(5,2), i NUMERIC(10,5),
|j MONEY, k SMALLMONEY)
""".stripMargin).executeUpdate()
conn.prepareStatement(
"""
|INSERT INTO numbers VALUES (
|0,
|255, 32767, 2147483647, 9223372036854775807,
|123456789012345.123456789012345, 123456789012345.123456789012345,
|123456789012345.123456789012345,
|123, 12345.12,
|922337203685477.58, 214748.3647)
""".stripMargin).executeUpdate()

conn.prepareStatement(
"""
|CREATE TABLE dates (
|a DATE, b DATETIME, c DATETIME2,
|d DATETIMEOFFSET, e SMALLDATETIME,
|f TIME)
""".stripMargin).executeUpdate()
conn.prepareStatement(
"""
|INSERT INTO dates VALUES (
|'1991-11-09', '1999-01-01 13:23:35', '9999-12-31 23:59:59',
|'1901-05-09 23:59:59 +14:00', '1996-01-01 23:23:45',
|'13:31:24')
""".stripMargin).executeUpdate()

conn.prepareStatement(
"""
|CREATE TABLE strings (
|a CHAR(10), b VARCHAR(10),
|c NCHAR(10), d NVARCHAR(10),
|e BINARY(4), f VARBINARY(4),
|g TEXT, h NTEXT,
|i IMAGE)
""".stripMargin).executeUpdate()
conn.prepareStatement(
"""
|INSERT INTO strings VALUES (
|'the', 'quick',
|'brown', 'fox',
|123456, 123456,
|'the', 'lazy',
|'dog')
""".stripMargin).executeUpdate()
}

test("Basic test") {
val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties)
val rows = df.collect()
assert(rows.length == 2)
val types = rows(0).toSeq.map(x => x.getClass.toString)
assert(types.length == 2)
assert(types(0).equals("class java.lang.Integer"))
assert(types(1).equals("class java.lang.String"))
}

test("Numeric types") {
val df = spark.read.jdbc(jdbcUrl, "numbers", new Properties)
val rows = df.collect()
assert(rows.length == 1)
val row = rows(0)
val types = row.toSeq.map(x => x.getClass.toString)
assert(types.length == 12)
assert(types(0).equals("class java.lang.Boolean"))
assert(types(1).equals("class java.lang.Integer"))
assert(types(2).equals("class java.lang.Short"))
assert(types(3).equals("class java.lang.Integer"))
assert(types(4).equals("class java.lang.Long"))
assert(types(5).equals("class java.lang.Double"))
assert(types(6).equals("class java.lang.Float"))
assert(types(7).equals("class java.lang.Float"))
assert(types(8).equals("class java.math.BigDecimal"))
assert(types(9).equals("class java.math.BigDecimal"))
assert(types(10).equals("class java.math.BigDecimal"))
assert(types(11).equals("class java.math.BigDecimal"))
assert(row.getBoolean(0) == false)
assert(row.getInt(1) == 255)
assert(row.getShort(2) == 32767)
assert(row.getInt(3) == 2147483647)
assert(row.getLong(4) == 9223372036854775807L)
assert(row.getDouble(5) == 1.2345678901234512E14) // float = float(53) has 15-digits precision
assert(row.getFloat(6) == 1.23456788103168E14) // float(24) has 7-digits precision
assert(row.getFloat(7) == 1.23456788103168E14) // real = float(24)
assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00")))
assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000")))
assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800")))
assert(row.getAs[BigDecimal](11).equals(new BigDecimal("214748.3647")))
}

test("Date types") {
val df = spark.read.jdbc(jdbcUrl, "dates", new Properties)
val rows = df.collect()
assert(rows.length == 1)
val row = rows(0)
val types = row.toSeq.map(x => x.getClass.toString)
assert(types.length == 6)
assert(types(0).equals("class java.sql.Date"))
assert(types(1).equals("class java.sql.Timestamp"))
assert(types(2).equals("class java.sql.Timestamp"))
assert(types(3).equals("class java.lang.String"))
assert(types(4).equals("class java.sql.Timestamp"))
assert(types(5).equals("class java.sql.Timestamp"))
assert(row.getAs[Date](0).equals(Date.valueOf("1991-11-09")))
assert(row.getAs[Timestamp](1).equals(Timestamp.valueOf("1999-01-01 13:23:35.0")))
assert(row.getAs[Timestamp](2).equals(Timestamp.valueOf("9999-12-31 23:59:59.0")))
assert(row.getString(3).equals("1901-05-09 23:59:59.0000000 +14:00"))
assert(row.getAs[Timestamp](4).equals(Timestamp.valueOf("1996-01-01 23:24:00.0")))
assert(row.getAs[Timestamp](5).equals(Timestamp.valueOf("1900-01-01 13:31:24.0")))
}

test("String types") {
val df = spark.read.jdbc(jdbcUrl, "strings", new Properties)
val rows = df.collect()
assert(rows.length == 1)
val row = rows(0)
val types = row.toSeq.map(x => x.getClass.toString)
assert(types.length == 9)
assert(types(0).equals("class java.lang.String"))
assert(types(1).equals("class java.lang.String"))
assert(types(2).equals("class java.lang.String"))
assert(types(3).equals("class java.lang.String"))
assert(types(4).equals("class [B"))
assert(types(5).equals("class [B"))
assert(types(6).equals("class java.lang.String"))
assert(types(7).equals("class java.lang.String"))
assert(types(8).equals("class [B"))
assert(row.getString(0).length == 10)
assert(row.getString(0).trim.equals("the"))
assert(row.getString(1).equals("quick"))
assert(row.getString(2).length == 10)
assert(row.getString(2).trim.equals("brown"))
assert(row.getString(3).equals("fox"))
assert(java.util.Arrays.equals(row.getAs[Array[Byte]](4), Array[Byte](0, 1, -30, 64)))
assert(java.util.Arrays.equals(row.getAs[Array[Byte]](5), Array[Byte](0, 1, -30, 64)))
assert(row.getString(6).equals("the"))
assert(row.getString(7).equals("lazy"))
assert(java.util.Arrays.equals(row.getAs[Array[Byte]](8), Array[Byte](100, 111, 103)))
}

test("Basic write test") {
val df1 = spark.read.jdbc(jdbcUrl, "numbers", new Properties)
val df2 = spark.read.jdbc(jdbcUrl, "dates", new Properties)
val df3 = spark.read.jdbc(jdbcUrl, "strings", new Properties)
df1.write.jdbc(jdbcUrl, "numberscopy", new Properties)
df2.write.jdbc(jdbcUrl, "datescopy", new Properties)
df3.write.jdbc(jdbcUrl, "stringscopy", new Properties)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,20 @@ private object MsSqlServerDialect extends JdbcDialect {
// String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients
Option(StringType)
} else {
None
sqlType match {
case java.sql.Types.SMALLINT => Some(ShortType)
case java.sql.Types.REAL => Some(FloatType)
case _ => None
}
}
}

override def getJDBCType(dt: DataType): Option[JdbcType] = dt match {
case TimestampType => Some(JdbcType("DATETIME", java.sql.Types.TIMESTAMP))
case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR))
case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT))
case BinaryType => Some(JdbcType("VARBINARY(MAX)", java.sql.Types.VARBINARY))
case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT))
case _ => None
}

Expand Down
23 changes: 23 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -860,6 +860,29 @@ class JDBCSuite extends QueryTest
Some(TimestampType))
}

test("MsSqlServerDialect jdbc type mapping") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay. Since this is the same with master, we can ignore adding JIRA ID.

Copy link
Contributor Author

@shivsood shivsood Jul 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you referring to "test("MsSqlServerDialect jdbc type mapping")"? The fix updated this function, did not create a new function. For the new function that i added, i have mentioned the JIRA ID.

Copy link
Member

@dongjoon-hyun dongjoon-hyun Jul 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's exactly what I meant, @shivsood . The above comment is not about requesting changes. It was supporting your code. Usually, reviewers leave their comments for this other reviewers.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks

val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver")
assert(msSqlServerDialect.getJDBCType(TimestampType).map(_.databaseTypeDefinition).get ==
"DATETIME")
assert(msSqlServerDialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get ==
"NVARCHAR(MAX)")
assert(msSqlServerDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get ==
"BIT")
assert(msSqlServerDialect.getJDBCType(BinaryType).map(_.databaseTypeDefinition).get ==
"VARBINARY(MAX)")
assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get ==
"SMALLINT")
}

test("SPARK-28152 MsSqlServerDialect catalyst type mapping") {
val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver")
val metadata = new MetadataBuilder().putLong("scale", 1)
assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1,
metadata).get == ShortType)
assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1,
metadata).get == FloatType)
}

test("table exists query by jdbc dialect") {
val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db")
val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db")
Expand Down