Skip to content

Commit cafca54

Browse files
committed
[SPARK-20557][SQL] Support JDBC data type Time with Time Zone
### What changes were proposed in this pull request? This PR is to support JDBC data type TIME WITH TIME ZONE. It can be converted to TIMESTAMP In addition, before this PR, for unsupported data types, we simply output the type number instead of the type name. ``` java.sql.SQLException: Unsupported type 2014 ``` After this PR, the message is like ``` java.sql.SQLException: Unsupported type TIMESTAMP_WITH_TIMEZONE ``` - Also upgrade the H2 version to `1.4.195` which has the type fix for "TIMESTAMP WITH TIMEZONE". However, it is not fully supported. Thus, we capture the exception, but we still need it to partially test the support of "TIMESTAMP WITH TIMEZONE", because Docker tests are not regularly run. ### How was this patch tested? Added test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17835 from gatorsmile/h2.
1 parent b433aca commit cafca54

File tree

6 files changed

+48
-8
lines changed

6 files changed

+48
-8
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -192,7 +192,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo
192192
checkRow(sql("SELECT * FROM datetime1 where id = 1").head())
193193
}
194194

195-
test("SPARK-20557: column type TIMEZONE with TIME STAMP should be recognized") {
195+
test("SPARK-20557: column type TIMESTAMP with TIME ZONE should be recognized") {
196196
val dfRead = sqlContext.read.jdbc(jdbcUrl, "ts_with_timezone", new Properties)
197197
val rows = dfRead.collect()
198198
val types = rows(0).toSeq.map(x => x.getClass.toString)

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

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,13 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
5555
+ "null, null, null, null, null, "
5656
+ "null, null, null, null, null, null, null)"
5757
).executeUpdate()
58+
59+
conn.prepareStatement("CREATE TABLE ts_with_timezone " +
60+
"(id integer, tstz TIMESTAMP WITH TIME ZONE, ttz TIME WITH TIME ZONE)")
61+
.executeUpdate()
62+
conn.prepareStatement("INSERT INTO ts_with_timezone VALUES " +
63+
"(1, TIMESTAMP WITH TIME ZONE '2016-08-12 10:22:31.949271-07', TIME WITH TIME ZONE '17:22:31.949271+00')")
64+
.executeUpdate()
5865
}
5966

6067
test("Type mapping for various types") {
@@ -126,4 +133,12 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
126133
assert(schema(0).dataType == FloatType)
127134
assert(schema(1).dataType == ShortType)
128135
}
136+
137+
test("SPARK-20557: column type TIMESTAMP with TIME ZONE and TIME with TIME ZONE should be recognized") {
138+
val dfRead = sqlContext.read.jdbc(jdbcUrl, "ts_with_timezone", new Properties)
139+
val rows = dfRead.collect()
140+
val types = rows(0).toSeq.map(x => x.getClass.toString)
141+
assert(types(1).equals("class java.sql.Timestamp"))
142+
assert(types(2).equals("class java.sql.Timestamp"))
143+
}
129144
}

sql/core/pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,7 @@
115115
<dependency>
116116
<groupId>com.h2database</groupId>
117117
<artifactId>h2</artifactId>
118-
<version>1.4.183</version>
118+
<version>1.4.195</version>
119119
<scope>test</scope>
120120
</dependency>
121121
<dependency>

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.execution.datasources.jdbc
1919

20-
import java.sql.{Connection, Driver, DriverManager, PreparedStatement, ResultSet, ResultSetMetaData, SQLException}
20+
import java.sql.{Connection, Driver, DriverManager, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException}
2121
import java.util.Locale
2222

2323
import scala.collection.JavaConverters._
@@ -217,23 +217,29 @@ object JdbcUtils extends Logging {
217217
case java.sql.Types.OTHER => null
218218
case java.sql.Types.REAL => DoubleType
219219
case java.sql.Types.REF => StringType
220+
case java.sql.Types.REF_CURSOR => null
220221
case java.sql.Types.ROWID => LongType
221222
case java.sql.Types.SMALLINT => IntegerType
222223
case java.sql.Types.SQLXML => StringType
223224
case java.sql.Types.STRUCT => StringType
224225
case java.sql.Types.TIME => TimestampType
226+
case java.sql.Types.TIME_WITH_TIMEZONE
227+
=> TimestampType
225228
case java.sql.Types.TIMESTAMP => TimestampType
226229
case java.sql.Types.TIMESTAMP_WITH_TIMEZONE
227230
=> TimestampType
228231
case -101 => TimestampType // Value for Timestamp with Time Zone in Oracle
229232
case java.sql.Types.TINYINT => IntegerType
230233
case java.sql.Types.VARBINARY => BinaryType
231234
case java.sql.Types.VARCHAR => StringType
232-
case _ => null
235+
case _ =>
236+
throw new SQLException("Unrecognized SQL type " + sqlType)
233237
// scalastyle:on
234238
}
235239

236-
if (answer == null) throw new SQLException("Unsupported type " + sqlType)
240+
if (answer == null) {
241+
throw new SQLException("Unsupported type " + JDBCType.valueOf(sqlType).getName)
242+
}
237243
answer
238244
}
239245

sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@ import org.apache.spark.sql.types.StructType
3333
import org.apache.spark.storage.StorageLevel
3434

3535

36-
3736
/**
3837
* Internal implementation of the user-facing `Catalog`.
3938
*/

sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,13 +18,13 @@
1818
package org.apache.spark.sql.jdbc
1919

2020
import java.math.BigDecimal
21-
import java.sql.{Date, DriverManager, Timestamp}
21+
import java.sql.{Date, DriverManager, SQLException, Timestamp}
2222
import java.util.{Calendar, GregorianCalendar, Properties}
2323

2424
import org.h2.jdbc.JdbcSQLException
2525
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
2626

27-
import org.apache.spark.SparkFunSuite
27+
import org.apache.spark.{SparkException, SparkFunSuite}
2828
import org.apache.spark.sql.{AnalysisException, DataFrame, Row}
2929
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
3030
import org.apache.spark.sql.execution.DataSourceScanExec
@@ -141,6 +141,15 @@ class JDBCSuite extends SparkFunSuite
141141
|OPTIONS (url '$url', dbtable 'TEST.TIMETYPES', user 'testUser', password 'testPass')
142142
""".stripMargin.replaceAll("\n", " "))
143143

144+
conn.prepareStatement("CREATE TABLE test.timezone (tz TIMESTAMP WITH TIME ZONE) " +
145+
"AS SELECT '1999-01-08 04:05:06.543543543 GMT-08:00'")
146+
.executeUpdate()
147+
conn.commit()
148+
149+
conn.prepareStatement("CREATE TABLE test.array (ar ARRAY) " +
150+
"AS SELECT '(1, 2, 3)'")
151+
.executeUpdate()
152+
conn.commit()
144153

145154
conn.prepareStatement("create table test.flttypes (a DOUBLE, b REAL, c DECIMAL(38, 18))"
146155
).executeUpdate()
@@ -919,6 +928,17 @@ class JDBCSuite extends SparkFunSuite
919928
assert(res === (foobarCnt, 0L, foobarCnt) :: Nil)
920929
}
921930

931+
test("unsupported types") {
932+
var e = intercept[SparkException] {
933+
spark.read.jdbc(urlWithUserAndPass, "TEST.TIMEZONE", new Properties()).collect()
934+
}.getMessage
935+
assert(e.contains("java.lang.UnsupportedOperationException: unimplemented"))
936+
e = intercept[SQLException] {
937+
spark.read.jdbc(urlWithUserAndPass, "TEST.ARRAY", new Properties()).collect()
938+
}.getMessage
939+
assert(e.contains("Unsupported type ARRAY"))
940+
}
941+
922942
test("SPARK-19318: Connection properties keys should be case-sensitive.") {
923943
def testJdbcOptions(options: JDBCOptions): Unit = {
924944
// Spark JDBC data source options are case-insensitive

0 commit comments

Comments
 (0)