Skip to content
Open
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,10 @@

package org.apache.spark.sql.connect.client.jdbc

import java.sql.{ResultSet, SQLException, Types}
import java.math.{BigDecimal => JBigDecimal}
import java.nio.charset.StandardCharsets
import java.sql.{Date, ResultSet, SQLException, Time, Timestamp, Types}
import java.util.{Calendar, TimeZone}

import scala.util.Using

Expand Down Expand Up @@ -236,7 +239,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
val decimalType = s"DECIMAL($precision,$scale)"
withExecuteQuery(stmt, s"SELECT cast('$value' as $decimalType)") { rs =>
assert(rs.next())
assert(rs.getBigDecimal(1) === new java.math.BigDecimal(value))
assert(rs.getBigDecimal(1) === new JBigDecimal(value))
assert(!rs.wasNull)
assert(!rs.next())

Expand Down Expand Up @@ -300,14 +303,14 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
("cast(1 AS FLOAT)", (rs: ResultSet) => rs.getFloat(1), 1.toFloat),
("cast(1 AS DOUBLE)", (rs: ResultSet) => rs.getDouble(1), 1.toDouble),
("cast(1 AS DECIMAL(10,5))", (rs: ResultSet) => rs.getBigDecimal(1),
new java.math.BigDecimal("1.00000")),
new JBigDecimal("1.00000")),
("CAST(X'0A0B0C' AS BINARY)", (rs: ResultSet) => rs.getBytes(1),
Array[Byte](0x0A, 0x0B, 0x0C)),
("date '2023-11-15'", (rs: ResultSet) => rs.getDate(1),
java.sql.Date.valueOf("2023-11-15")),
Date.valueOf("2023-11-15")),
("time '12:34:56.123456'", (rs: ResultSet) => rs.getTime(1), {
val millis = timeToMillis(12, 34, 56, 123)
new java.sql.Time(millis)
new Time(millis)
})
).foreach {
case (query, getter, expectedValue) =>
Expand Down Expand Up @@ -336,7 +339,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
// Test basic date type
withExecuteQuery(stmt, "SELECT date '2023-11-15'") { rs =>
assert(rs.next())
assert(rs.getDate(1) === java.sql.Date.valueOf("2023-11-15"))
assert(rs.getDate(1) === Date.valueOf("2023-11-15"))
assert(!rs.wasNull)
assert(!rs.next())

Expand Down Expand Up @@ -376,7 +379,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
// Test date type by column label
withExecuteQuery(stmt, "SELECT date '2025-11-15' as test_date") { rs =>
assert(rs.next())
assert(rs.getDate("test_date") === java.sql.Date.valueOf("2025-11-15"))
assert(rs.getDate("test_date") === Date.valueOf("2025-11-15"))
assert(!rs.wasNull)
assert(!rs.next())
}
Expand All @@ -397,7 +400,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
assert(!rs.wasNull)

val stringValue = rs.getString(1)
val expectedString = new String(testBytes, java.nio.charset.StandardCharsets.UTF_8)
val expectedString = new String(testBytes, StandardCharsets.UTF_8)
assert(stringValue === expectedString)

assert(!rs.next())
Expand All @@ -411,7 +414,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
}

// Test binary type with UTF-8 text
val textBytes = "\\xDeAdBeEf".getBytes(java.nio.charset.StandardCharsets.UTF_8)
val textBytes = "\\xDeAdBeEf".getBytes(StandardCharsets.UTF_8)
val hexString2 = textBytes.map(b => "%02X".format(b)).mkString
withExecuteQuery(stmt, s"SELECT CAST(X'$hexString2' AS BINARY)") { rs =>
assert(rs.next())
Expand Down Expand Up @@ -449,6 +452,11 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
assert(bytes.length === testBytes2.length)
assert(bytes.sameElements(testBytes2))
assert(!rs.wasNull)

val stringValue = rs.getString("test_binary")
Copy link
Contributor

Choose a reason for hiding this comment

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

I think these 3 lines would be fine.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I restore to only leave these three lines.

val expectedString = new String(testBytes2, StandardCharsets.UTF_8)
assert(stringValue === expectedString)

assert(!rs.next())

val metaData = rs.getMetaData
Expand Down Expand Up @@ -580,7 +588,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
stmt.execute(s"set spark.sql.datetime.java8API.enabled=$java8APIEnabled")
Using.resource(stmt.executeQuery("SELECT date '2025-11-15'")) { rs =>
assert(rs.next())
assert(rs.getDate(1) === java.sql.Date.valueOf("2025-11-15"))
assert(rs.getDate(1) === Date.valueOf("2025-11-15"))
assert(!rs.wasNull)
assert(!rs.next())
}
Expand Down Expand Up @@ -611,7 +619,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
assert(rs.next())
val timestamp = rs.getTimestamp(1)
assert(timestamp !== null)
assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(timestamp === Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(!rs.wasNull)
assert(!rs.next())

Expand Down Expand Up @@ -656,18 +664,18 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
// Test by column label
val timestamp = rs.getTimestamp("test_timestamp")
assert(timestamp !== null)
assert(timestamp === java.sql.Timestamp.valueOf(tsString))
assert(timestamp === Timestamp.valueOf(tsString))
assert(!rs.wasNull)

// Test with calendar - should return same value (Calendar is ignored)
// Note: Spark Connect handles timezone at server, Calendar param is for API compliance
val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC"))
val calUTC = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
val timestampUTC = rs.getTimestamp(1, calUTC)
assert(timestampUTC !== null)
assert(timestampUTC.getTime === timestamp.getTime)

val calPST = java.util.Calendar.getInstance(
java.util.TimeZone.getTimeZone("America/Los_Angeles"))
val calPST = Calendar.getInstance(
TimeZone.getTimeZone("America/Los_Angeles"))
val timestampPST = rs.getTimestamp(1, calPST)
assert(timestampPST !== null)
// Same value regardless of calendar
Expand All @@ -692,7 +700,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
assert(rs.next())

// Calendar parameter should not affect null handling
val cal = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC"))
val cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
val timestamp = rs.getTimestamp(1, cal)
assert(timestamp === null)
assert(rs.wasNull)
Expand All @@ -708,7 +716,7 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
assert(rs.next())
val timestamp = rs.getTimestamp(1)
assert(timestamp !== null)
assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(timestamp === Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(!rs.wasNull)
assert(!rs.next())

Expand All @@ -733,11 +741,11 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
// Test by column label
val timestamp = rs.getTimestamp("test_ts_ntz")
assert(timestamp !== null)
assert(timestamp === java.sql.Timestamp.valueOf(tsString))
assert(timestamp === Timestamp.valueOf(tsString))
assert(!rs.wasNull)

// Test with calendar - should return same value (Calendar is ignored)
val calUTC = java.util.Calendar.getInstance(java.util.TimeZone.getTimeZone("UTC"))
val calUTC = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
val timestampCal = rs.getTimestamp(1, calUTC)
assert(timestampCal !== null)
assert(timestampCal.getTime === timestamp.getTime)
Expand Down Expand Up @@ -770,13 +778,13 @@ class SparkConnectJdbcDataTypeSuite extends ConnectFunSuite with RemoteSparkSess
// Test TIMESTAMP type
val timestamp = rs.getTimestamp(1)
assert(timestamp !== null)
assert(timestamp === java.sql.Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(timestamp === Timestamp.valueOf("2025-11-15 10:30:45.123456"))
assert(!rs.wasNull)

// Test TIMESTAMP_NTZ type
val timestampNtz = rs.getTimestamp(2)
assert(timestampNtz !== null)
assert(timestampNtz === java.sql.Timestamp.valueOf("2025-11-15 14:22:33.789012"))
assert(timestampNtz === Timestamp.valueOf("2025-11-15 14:22:33.789012"))
assert(!rs.wasNull)

assert(!rs.next())
Expand Down