You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "sadikovi (via GitHub)" <gi...@apache.org> on 2023/04/05 22:21:49 UTC

[GitHub] [spark] sadikovi commented on a diff in pull request #40678: [SPARK-43040][SQL] Improve TimestampNTZ type support in JDBC data source

sadikovi commented on code in PR #40678:
URL: https://github.com/apache/spark/pull/40678#discussion_r1159080729


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala:
##########
@@ -328,21 +328,25 @@ object JdbcUtils extends Logging with SQLConfHelper {
   /**
    * Convert a [[ResultSet]] into an iterator of Catalyst Rows.
    */
-  def resultSetToRows(resultSet: ResultSet, schema: StructType): Iterator[Row] = {
+  def resultSetToRows(
+      resultSet: ResultSet,
+      schema: StructType,
+      dialect: Option[JdbcDialect] = None): Iterator[Row] = {

Review Comment:
   Why is dialect optional? I think you can just pass dialect as JdbcDialect.



##########
connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala:
##########
@@ -379,4 +386,32 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite {
     assert(row(0).length === 1)
     assert(row(0).getString(0) === "$1,000.00")
   }
+
+  test("SPARK-43040: timestamp_ntz read test") {
+    val prop = new Properties
+    prop.setProperty("preferTimestampNTZ", "true")
+    val df = sqlContext.read.jdbc(jdbcUrl, "timestamp_ntz", prop)
+    val row = df.collect()
+    assert(row.length === 3)
+    assert(row(0).length === 1)
+    assert(row(0) === Row(LocalDateTime.of(2013, 4, 5, 12, 1, 2)))

Review Comment:
   What result do you get without these changes?



##########
sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala:
##########
@@ -98,6 +100,14 @@ private object PostgresDialect extends JdbcDialect with SQLConfHelper {
     case _ => None
   }
 
+  override def convertJavaTimestampToTimestampNTZ(t: Timestamp): Long = {
+    DateTimeUtils.localDateTimeToMicros(t.toLocalDateTime)
+  }
+
+  override  def convertTimestampNTZToJavaTimestamp(ldt: LocalDateTime): Timestamp = {

Review Comment:
   nit: extra space should be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org