Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
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 @@ -18,22 +18,50 @@
*/
package org.apache.sedona.sql.datasources.geopackage.transform

import java.time.{Instant, LocalDate}
import java.time.{Instant, LocalDate, LocalDateTime, ZoneOffset}
import java.time.format.DateTimeFormatter
import java.time.format.DateTimeParseException
import java.time.temporal.ChronoUnit

object DataTypesTransformations {
def getDays(dateString: String): Int = {
val formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd")

val date = LocalDate.parse(dateString, formatter)
// Pre-created formatters to avoid repeated object creation
private val dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd")
private val datetimeFormatters = Array(
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS"), // 3 digits
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SS"), // 2 digits
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.S"), // 1 digit
DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss") // no milliseconds
)

def getDays(dateString: String): Int = {
val date = LocalDate.parse(dateString, dateFormatter)
val epochDate = LocalDate.of(1970, 1, 1)

ChronoUnit.DAYS.between(epochDate, date).toInt
}

def epoch(timestampStr: String): Long = {
Instant.parse(timestampStr).toEpochMilli
try {
// Try parsing as-is first (works for timestamps with timezone info)
Instant.parse(timestampStr).toEpochMilli
} catch {
case _: DateTimeParseException =>
// If parsing fails, try treating it as UTC (common case for GeoPackage)
// Handle various datetime formats without timezone info
// Try different patterns to handle various millisecond formats
for (formatter <- datetimeFormatters) {
try {
val localDateTime = LocalDateTime.parse(timestampStr, formatter)
return localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli
} catch {
case _: DateTimeParseException =>
// Continue to next formatter
}
}

// If all formatters failed, throw a descriptive exception
throw new IllegalArgumentException(
s"Unable to parse datetime: $timestampStr. " +
s"Expected formats: 'yyyy-MM-ddTHH:mm:ss[.S]' or 'yyyy-MM-ddTHH:mm:ss[.S]Z'")
Copy link
Preview

Copilot AI Sep 7, 2025

Choose a reason for hiding this comment

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

[nitpick] Using early return in a for loop is not optimal for functional programming. Consider using datetimeFormatters.view.map(...) with find or collectFirst to make the code more functional and potentially more efficient.

Suggested change
for (formatter <- datetimeFormatters) {
try {
val localDateTime = LocalDateTime.parse(timestampStr, formatter)
return localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli
} catch {
case _: DateTimeParseException =>
// Continue to next formatter
}
}
// If all formatters failed, throw a descriptive exception
throw new IllegalArgumentException(
s"Unable to parse datetime: $timestampStr. " +
s"Expected formats: 'yyyy-MM-ddTHH:mm:ss[.S]' or 'yyyy-MM-ddTHH:mm:ss[.S]Z'")
datetimeFormatters.view
.map { formatter =>
try {
Some(LocalDateTime.parse(timestampStr, formatter).toInstant(ZoneOffset.UTC).toEpochMilli)
} catch {
case _: DateTimeParseException => None
}
}
.collectFirst { case Some(epochMilli) => epochMilli }
.getOrElse {
// If all formatters failed, throw a descriptive exception
throw new IllegalArgumentException(
s"Unable to parse datetime: $timestampStr. " +
s"Expected formats: 'yyyy-MM-ddTHH:mm:ss[.S]' or 'yyyy-MM-ddTHH:mm:ss[.S]Z'")
}

Copilot uses AI. Check for mistakes.

Copy link
Preview

Copilot AI Sep 7, 2025

Choose a reason for hiding this comment

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

The error message format examples don't accurately reflect all supported patterns. It shows '[.S]' but the code supports .S, .SS, and .SSS formats. Consider updating to: 'yyyy-MM-ddTHH:mm:ss[.SSS|.SS|.S]' or 'yyyy-MM-ddTHH:mm:ss[.SSS|.SS|.S]Z'

Suggested change
s"Expected formats: 'yyyy-MM-ddTHH:mm:ss[.S]' or 'yyyy-MM-ddTHH:mm:ss[.S]Z'")
s"Expected formats: 'yyyy-MM-ddTHH:mm:ss[.SSS|.SS|.S]' or 'yyyy-MM-ddTHH:mm:ss[.SSS|.SS|.S]Z'")

Copilot uses AI. Check for mistakes.

}
}
}
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@
*/
package org.apache.sedona.sql

import io.minio.{MakeBucketArgs, MinioClient, PutObjectArgs}
import org.apache.spark.sql.{DataFrame, SparkSession}
import io.minio.{MakeBucketArgs, MinioClient}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.functions.expr
import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
import org.apache.spark.sql.types.{BinaryType, BooleanType, DateType, DoubleType, IntegerType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.types._
import org.scalatest.matchers.should.Matchers
import org.scalatest.prop.TableDrivenPropertyChecks._
import org.testcontainers.containers.MinIOContainer
Expand Down Expand Up @@ -168,6 +168,50 @@ class GeoPackageReaderTest extends TestBaseScala with Matchers {
df.count() shouldEqual expectedCount
}
}

it("should handle datetime fields without timezone information") {
// This test verifies the fix for DateTimeParseException when reading
// GeoPackage files with datetime fields that don't include timezone info
val testFilePath = resourceFolder + "geopackage/test_datetime_issue.gpkg"

// Test reading the test_features table with problematic datetime formats
val df = sparkSession.read
.format("geopackage")
.option("tableName", "test_features")
.load(testFilePath)

// The test should not throw DateTimeParseException when reading datetime fields
noException should be thrownBy {
df.select("created_at", "updated_at").collect()
}

// Verify that datetime fields are properly parsed as TimestampType
df.schema.fields.find(_.name == "created_at").get.dataType shouldEqual TimestampType
df.schema.fields.find(_.name == "updated_at").get.dataType shouldEqual TimestampType

// Verify that we can read the datetime values
val datetimeValues = df.select("created_at", "updated_at").collect()
datetimeValues should not be empty

// Verify that datetime values are valid timestamps
datetimeValues.foreach { row =>
val createdTimestamp = row.getAs[Timestamp]("created_at")
val updatedTimestamp = row.getAs[Timestamp]("updated_at")
createdTimestamp should not be null
updatedTimestamp should not be null
createdTimestamp.getTime should be > 0L
updatedTimestamp.getTime should be > 0L
}

// Test showMetadata option with the same file
noException should be thrownBy {
val metadataDf = sparkSession.read
.format("geopackage")
.option("showMetadata", "true")
.load(testFilePath)
metadataDf.select("last_change").collect()
}
}
}

describe("GeoPackage Raster Data Test") {
Expand Down Expand Up @@ -257,7 +301,7 @@ class GeoPackageReaderTest extends TestBaseScala with Matchers {
.load(inputPath)
.count shouldEqual 34

val df = sparkSessionMinio.read
val df = sparkSession.read
.format("geopackage")
.option("tableName", "point1")
.load(inputPath)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,50 @@ class GeoPackageReaderTest extends TestBaseScala with Matchers {
df.count() shouldEqual expectedCount
}
}

it("should handle datetime fields without timezone information") {
// This test verifies the fix for DateTimeParseException when reading
// GeoPackage files with datetime fields that don't include timezone info
val testFilePath = resourceFolder + "geopackage/test_datetime_issue.gpkg"

// Test reading the test_features table with problematic datetime formats
val df = sparkSession.read
.format("geopackage")
.option("tableName", "test_features")
.load(testFilePath)

// The test should not throw DateTimeParseException when reading datetime fields
noException should be thrownBy {
df.select("created_at", "updated_at").collect()
}

// Verify that datetime fields are properly parsed as TimestampType
df.schema.fields.find(_.name == "created_at").get.dataType shouldEqual TimestampType
df.schema.fields.find(_.name == "updated_at").get.dataType shouldEqual TimestampType

// Verify that we can read the datetime values
val datetimeValues = df.select("created_at", "updated_at").collect()
datetimeValues should not be empty

// Verify that datetime values are valid timestamps
datetimeValues.foreach { row =>
val createdTimestamp = row.getAs[Timestamp]("created_at")
val updatedTimestamp = row.getAs[Timestamp]("updated_at")
createdTimestamp should not be null
updatedTimestamp should not be null
createdTimestamp.getTime should be > 0L
updatedTimestamp.getTime should be > 0L
}

// Test showMetadata option with the same file
noException should be thrownBy {
val metadataDf = sparkSession.read
.format("geopackage")
.option("showMetadata", "true")
.load(testFilePath)
metadataDf.select("last_change").collect()
}
}
}

describe("GeoPackage Raster Data Test") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,50 @@ class GeoPackageReaderTest extends TestBaseScala with Matchers {
df.count() shouldEqual expectedCount
}
}

it("should handle datetime fields without timezone information") {
// This test verifies the fix for DateTimeParseException when reading
// GeoPackage files with datetime fields that don't include timezone info
val testFilePath = resourceFolder + "geopackage/test_datetime_issue.gpkg"

// Test reading the test_features table with problematic datetime formats
val df = sparkSession.read
.format("geopackage")
.option("tableName", "test_features")
.load(testFilePath)

// The test should not throw DateTimeParseException when reading datetime fields
noException should be thrownBy {
df.select("created_at", "updated_at").collect()
}

// Verify that datetime fields are properly parsed as TimestampType
df.schema.fields.find(_.name == "created_at").get.dataType shouldEqual TimestampType
df.schema.fields.find(_.name == "updated_at").get.dataType shouldEqual TimestampType

// Verify that we can read the datetime values
val datetimeValues = df.select("created_at", "updated_at").collect()
datetimeValues should not be empty

// Verify that datetime values are valid timestamps
datetimeValues.foreach { row =>
val createdTimestamp = row.getAs[Timestamp]("created_at")
val updatedTimestamp = row.getAs[Timestamp]("updated_at")
createdTimestamp should not be null
updatedTimestamp should not be null
createdTimestamp.getTime should be > 0L
updatedTimestamp.getTime should be > 0L
}

// Test showMetadata option with the same file
noException should be thrownBy {
val metadataDf = sparkSession.read
.format("geopackage")
.option("showMetadata", "true")
.load(testFilePath)
metadataDf.select("last_change").collect()
}
}
}

describe("GeoPackage Raster Data Test") {
Expand Down
Loading