Skip to content

Commit 94de630

Browse files
lresendeJoshRosen
authored andcommitted
[SPARK-10521][SQL] Utilize Docker for test DB2 JDBC Dialect support
Add integration tests based on docker to test DB2 JDBC dialect support Author: Luciano Resende <[email protected]> Closes #9893 from lresende/SPARK-10521.
1 parent 3f0f408 commit 94de630

File tree

8 files changed

+215
-4
lines changed

8 files changed

+215
-4
lines changed

external/docker-integration-tests/pom.xml

Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,13 @@
3434
<sbt.project.name>docker-integration-tests</sbt.project.name>
3535
</properties>
3636

37+
<repositories>
38+
<repository>
39+
<id>db2</id>
40+
<url>https://app.camunda.com/nexus/content/repositories/public/</url>
41+
</repository>
42+
</repositories>
43+
3744
<dependencies>
3845
<dependency>
3946
<groupId>com.spotify</groupId>
@@ -180,5 +187,28 @@
180187
</exclusions>
181188
</dependency>
182189
<!-- End Jersey dependencies -->
190+
191+
<!-- DB2 JCC driver manual installation instructions
192+
193+
You can build this datasource if you:
194+
1) have the DB2 artifacts installed in a local repo and supply the URL:
195+
-Dmaven.repo.drivers=http://my.local.repo
196+
197+
2) have a copy of the DB2 JCC driver and run the following commands :
198+
mvn install:install-file -Dfile=${path to db2jcc4.jar} \
199+
-DgroupId=com.ibm.db2 \
200+
-DartifactId=db2jcc4 \
201+
-Dversion=10.5 \
202+
-Dpackaging=jar
203+
204+
Note: IBM DB2 JCC driver is available for download at
205+
http://www-01.ibm.com/support/docview.wss?uid=swg21363866
206+
-->
207+
<dependency>
208+
<groupId>com.ibm.db2.jcc</groupId>
209+
<artifactId>db2jcc4</artifactId>
210+
<version>10.5.0.5</version>
211+
<type>jar</type>
212+
</dependency>
183213
</dependencies>
184214
</project>
Lines changed: 157 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,157 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.jdbc
19+
20+
import java.math.BigDecimal
21+
import java.sql.{Connection, Date, Timestamp}
22+
import java.util.Properties
23+
24+
import org.scalatest._
25+
26+
import org.apache.spark.tags.DockerTest
27+
28+
@DockerTest
29+
@Ignore // AMPLab Jenkins needs to be updated before shared memory works on docker
30+
class DB2IntegrationSuite extends DockerJDBCIntegrationSuite {
31+
override val db = new DatabaseOnDocker {
32+
override val imageName = "lresende/db2express-c:10.5.0.5-3.10.0"
33+
override val env = Map(
34+
"DB2INST1_PASSWORD" -> "rootpass",
35+
"LICENSE" -> "accept"
36+
)
37+
override val usesIpc = true
38+
override val jdbcPort: Int = 50000
39+
override def getJdbcUrl(ip: String, port: Int): String =
40+
s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;"
41+
override def getStartupProcessName: Option[String] = Some("db2start")
42+
}
43+
44+
override def dataPreparation(conn: Connection): Unit = {
45+
conn.prepareStatement("CREATE TABLE tbl (x INTEGER, y VARCHAR(8))").executeUpdate()
46+
conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate()
47+
conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate()
48+
49+
conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits BIT(10), "
50+
+ "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, "
51+
+ "dbl DOUBLE)").executeUpdate()
52+
conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', "
53+
+ "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, "
54+
+ "42.75, 1.0000000000000002)").executeUpdate()
55+
56+
conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, "
57+
+ "yr YEAR)").executeUpdate()
58+
conn.prepareStatement("INSERT INTO dates VALUES ('1991-11-09', '13:31:24', "
59+
+ "'1996-01-01 01:23:45', '2009-02-13 23:31:30', '2001')").executeUpdate()
60+
61+
// TODO: Test locale conversion for strings.
62+
conn.prepareStatement("CREATE TABLE strings (a CHAR(10), b VARCHAR(10), c CLOB, d BLOB, "
63+
+ "e CHAR FOR BIT DATA)").executeUpdate()
64+
conn.prepareStatement("INSERT INTO strings VALUES ('the', 'quick', 'brown', 'fox', 'jumps'")
65+
.executeUpdate()
66+
}
67+
68+
test("Basic test") {
69+
val df = sqlContext.read.jdbc(jdbcUrl, "tbl", new Properties)
70+
val rows = df.collect()
71+
assert(rows.length == 2)
72+
val types = rows(0).toSeq.map(x => x.getClass.toString)
73+
assert(types.length == 2)
74+
assert(types(0).equals("class java.lang.Integer"))
75+
assert(types(1).equals("class java.lang.String"))
76+
}
77+
78+
test("Numeric types") {
79+
val df = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties)
80+
val rows = df.collect()
81+
assert(rows.length == 1)
82+
val types = rows(0).toSeq.map(x => x.getClass.toString)
83+
assert(types.length == 9)
84+
assert(types(0).equals("class java.lang.Boolean"))
85+
assert(types(1).equals("class java.lang.Long"))
86+
assert(types(2).equals("class java.lang.Integer"))
87+
assert(types(3).equals("class java.lang.Integer"))
88+
assert(types(4).equals("class java.lang.Integer"))
89+
assert(types(5).equals("class java.lang.Long"))
90+
assert(types(6).equals("class java.math.BigDecimal"))
91+
assert(types(7).equals("class java.lang.Double"))
92+
assert(types(8).equals("class java.lang.Double"))
93+
assert(rows(0).getBoolean(0) == false)
94+
assert(rows(0).getLong(1) == 0x225)
95+
assert(rows(0).getInt(2) == 17)
96+
assert(rows(0).getInt(3) == 77777)
97+
assert(rows(0).getInt(4) == 123456789)
98+
assert(rows(0).getLong(5) == 123456789012345L)
99+
val bd = new BigDecimal("123456789012345.12345678901234500000")
100+
assert(rows(0).getAs[BigDecimal](6).equals(bd))
101+
assert(rows(0).getDouble(7) == 42.75)
102+
assert(rows(0).getDouble(8) == 1.0000000000000002)
103+
}
104+
105+
test("Date types") {
106+
val df = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties)
107+
val rows = df.collect()
108+
assert(rows.length == 1)
109+
val types = rows(0).toSeq.map(x => x.getClass.toString)
110+
assert(types.length == 5)
111+
assert(types(0).equals("class java.sql.Date"))
112+
assert(types(1).equals("class java.sql.Timestamp"))
113+
assert(types(2).equals("class java.sql.Timestamp"))
114+
assert(types(3).equals("class java.sql.Timestamp"))
115+
assert(types(4).equals("class java.sql.Date"))
116+
assert(rows(0).getAs[Date](0).equals(Date.valueOf("1991-11-09")))
117+
assert(rows(0).getAs[Timestamp](1).equals(Timestamp.valueOf("1970-01-01 13:31:24")))
118+
assert(rows(0).getAs[Timestamp](2).equals(Timestamp.valueOf("1996-01-01 01:23:45")))
119+
assert(rows(0).getAs[Timestamp](3).equals(Timestamp.valueOf("2009-02-13 23:31:30")))
120+
assert(rows(0).getAs[Date](4).equals(Date.valueOf("2001-01-01")))
121+
}
122+
123+
test("String types") {
124+
val df = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties)
125+
val rows = df.collect()
126+
assert(rows.length == 1)
127+
val types = rows(0).toSeq.map(x => x.getClass.toString)
128+
assert(types.length == 9)
129+
assert(types(0).equals("class java.lang.String"))
130+
assert(types(1).equals("class java.lang.String"))
131+
assert(types(2).equals("class java.lang.String"))
132+
assert(types(3).equals("class java.lang.String"))
133+
assert(types(4).equals("class java.lang.String"))
134+
assert(types(5).equals("class java.lang.String"))
135+
assert(types(6).equals("class [B"))
136+
assert(types(7).equals("class [B"))
137+
assert(types(8).equals("class [B"))
138+
assert(rows(0).getString(0).equals("the"))
139+
assert(rows(0).getString(1).equals("quick"))
140+
assert(rows(0).getString(2).equals("brown"))
141+
assert(rows(0).getString(3).equals("fox"))
142+
assert(rows(0).getString(4).equals("jumps"))
143+
assert(rows(0).getString(5).equals("over"))
144+
assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](116, 104, 101, 0)))
145+
assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](7), Array[Byte](108, 97, 122, 121)))
146+
assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](8), Array[Byte](100, 111, 103)))
147+
}
148+
149+
test("Basic write test") {
150+
val df1 = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties)
151+
val df2 = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties)
152+
val df3 = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties)
153+
df1.write.jdbc(jdbcUrl, "numberscopy", new Properties)
154+
df2.write.jdbc(jdbcUrl, "datescopy", new Properties)
155+
df3.write.jdbc(jdbcUrl, "stringscopy", new Properties)
156+
}
157+
}

external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala

Lines changed: 18 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,11 @@ abstract class DatabaseOnDocker {
4444
*/
4545
val env: Map[String, String]
4646

47+
/**
48+
* Wheather or not to use ipc mode for shared memory when starting docker image
49+
*/
50+
val usesIpc: Boolean
51+
4752
/**
4853
* The container-internal JDBC port that the database listens on.
4954
*/
@@ -53,6 +58,11 @@ abstract class DatabaseOnDocker {
5358
* Return a JDBC URL that connects to the database running at the given IP address and port.
5459
*/
5560
def getJdbcUrl(ip: String, port: Int): String
61+
62+
/**
63+
* Optional process to run when container starts
64+
*/
65+
def getStartupProcessName: Option[String]
5666
}
5767

5868
abstract class DockerJDBCIntegrationSuite
@@ -97,17 +107,23 @@ abstract class DockerJDBCIntegrationSuite
97107
val dockerIp = DockerUtils.getDockerIp()
98108
val hostConfig: HostConfig = HostConfig.builder()
99109
.networkMode("bridge")
110+
.ipcMode(if (db.usesIpc) "host" else "")
100111
.portBindings(
101112
Map(s"${db.jdbcPort}/tcp" -> List(PortBinding.of(dockerIp, externalPort)).asJava).asJava)
102113
.build()
103114
// Create the database container:
104-
val config = ContainerConfig.builder()
115+
val containerConfigBuilder = ContainerConfig.builder()
105116
.image(db.imageName)
106117
.networkDisabled(false)
107118
.env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava)
108119
.hostConfig(hostConfig)
109120
.exposedPorts(s"${db.jdbcPort}/tcp")
110-
.build()
121+
if(db.getStartupProcessName.isDefined) {
122+
containerConfigBuilder
123+
.cmd(db.getStartupProcessName.get)
124+
}
125+
val config = containerConfigBuilder.build()
126+
// Create the database container:
111127
containerId = docker.createContainer(config).id
112128
// Start the container and wait until the database can accept JDBC connections:
113129
docker.startContainer(containerId)

external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,9 +30,11 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite {
3030
override val env = Map(
3131
"MYSQL_ROOT_PASSWORD" -> "rootpass"
3232
)
33+
override val usesIpc = false
3334
override val jdbcPort: Int = 3306
3435
override def getJdbcUrl(ip: String, port: Int): String =
3536
s"jdbc:mysql://$ip:$port/mysql?user=root&password=rootpass"
37+
override def getStartupProcessName: Option[String] = None
3638
}
3739

3840
override def dataPreparation(conn: Connection): Unit = {

external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,9 +52,11 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo
5252
override val env = Map(
5353
"ORACLE_ROOT_PASSWORD" -> "oracle"
5454
)
55+
override val usesIpc = false
5556
override val jdbcPort: Int = 1521
5657
override def getJdbcUrl(ip: String, port: Int): String =
5758
s"jdbc:oracle:thin:system/oracle@//$ip:$port/xe"
59+
override def getStartupProcessName: Option[String] = None
5860
}
5961

6062
override def dataPreparation(conn: Connection): Unit = {

external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,11 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
3232
override val env = Map(
3333
"POSTGRES_PASSWORD" -> "rootpass"
3434
)
35+
override val usesIpc = false
3536
override val jdbcPort = 5432
3637
override def getJdbcUrl(ip: String, port: Int): String =
3738
s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass"
39+
override def getStartupProcessName: Option[String] = None
3840
}
3941

4042
override def dataPreparation(conn: Connection): Unit = {

pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -666,7 +666,7 @@
666666
<groupId>com.spotify</groupId>
667667
<artifactId>docker-client</artifactId>
668668
<classifier>shaded</classifier>
669-
<version>3.4.0</version>
669+
<version>3.6.6</version>
670670
<scope>test</scope>
671671
<exclusions>
672672
<exclusion>

project/SparkBuild.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -366,8 +366,10 @@ object Flume {
366366
object DockerIntegrationTests {
367367
// This serves to override the override specified in DependencyOverrides:
368368
lazy val settings = Seq(
369-
dependencyOverrides += "com.google.guava" % "guava" % "18.0"
369+
dependencyOverrides += "com.google.guava" % "guava" % "18.0",
370+
resolvers ++= Seq("DB2" at "https://app.camunda.com/nexus/content/repositories/public/")
370371
)
372+
371373
}
372374

373375
/**

0 commit comments

Comments
 (0)