diff --git a/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java b/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java index 6840e8101..06d0bd95d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/DataTypeUtils.java @@ -4,18 +4,24 @@ import com.clickhouse.data.ClickHouseDataType; import java.time.Duration; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoField; import java.time.temporal.ChronoUnit; import java.time.temporal.TemporalUnit; +import java.util.Calendar; import java.util.Objects; +import java.util.TimeZone; import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES; @@ -47,6 +53,10 @@ public class DataTypeUtils { public static final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss"); + public static final DateTimeFormatter DATE_TIME_WITH_OPTIONAL_NANOS = new DateTimeFormatterBuilder().appendPattern("uuuu-MM-dd HH:mm:ss") + .appendOptional(new DateTimeFormatterBuilder().appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true).toFormatter()) + .toFormatter(); + /** * Formats an {@link Instant} object for use in SQL statements or as query * parameter. @@ -219,4 +229,134 @@ public static Duration localDateTimeToDuration(LocalDateTime localDateTime) { return Duration.ofSeconds(localDateTime.toEpochSecond(ZoneOffset.UTC)) .plusNanos(localDateTime.getNano()); } + + + /** + * Converts a {@link java.sql.Date} to {@link LocalDate} using the specified timezone. + * + *
For default JVM timezone behavior, use {@link Date#toLocalDate()} directly.
+ * + * @param sqlDate the java.sql.Date to convert + * @param timeZone the timezone context + * @return the LocalDate representing the date in the specified timezone + * @throws NullPointerException if sqlDate or timeZone is null + */ + public static LocalDate toLocalDate(Date sqlDate, TimeZone timeZone) { + Objects.requireNonNull(sqlDate, "sqlDate must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + ZoneId zoneId = timeZone.toZoneId(); + return Instant.ofEpochMilli(sqlDate.getTime()) + .atZone(zoneId) + .toLocalDate(); + } + + /** + * Converts a {@link java.sql.Time} to {@link LocalTime} using the specified timezone. + * + *For default JVM timezone behavior, use {@link Time#toLocalTime()} directly.
+ * + * @param sqlTime the java.sql.Time to convert + * @param timeZone the timezone context + * @return the LocalTime representing the time in the specified timezone + * @throws NullPointerException if sqlTime or timeZone is null + */ + public static LocalTime toLocalTime(Time sqlTime, TimeZone timeZone) { + Objects.requireNonNull(sqlTime, "sqlTime must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + ZoneId zoneId = timeZone.toZoneId(); + return Instant.ofEpochMilli(sqlTime.getTime()) + .atZone(zoneId) + .toLocalTime(); + } + + /** + * Converts a {@link java.sql.Timestamp} to {@link LocalDateTime} using the specified timezone. + * + *Note: This method preserves nanosecond precision from the Timestamp.
+ * + *For default JVM timezone behavior, use {@link Timestamp#toLocalDateTime()} directly.
+ * + * @param sqlTimestamp the java.sql.Timestamp to convert + * @param timeZone the timezone context + * @return the LocalDateTime representing the timestamp in the specified timezone + * @throws NullPointerException if sqlTimestamp or timeZone is null + */ + public static LocalDateTime toLocalDateTime(Timestamp sqlTimestamp, TimeZone timeZone) { + Objects.requireNonNull(sqlTimestamp, "sqlTimestamp must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + ZoneId zoneId = timeZone.toZoneId(); + // Use Instant to preserve nanoseconds + return LocalDateTime.ofInstant(sqlTimestamp.toInstant(), zoneId); + } + + // ==================== LocalDate/LocalTime/LocalDateTime to SQL types ==================== + + /** + * Converts a {@link LocalDate} to {@link java.sql.Date} using the specified timezone. + * + *For default JVM timezone behavior, use {@link Date#valueOf(LocalDate)} directly.
+ * + * @param localDate the LocalDate to convert + * @param timeZone the timezone context + * @return the java.sql.Date representing midnight on the specified date in the given timezone + * @throws NullPointerException if localDate or timeZone is null + */ + public static Date toSqlDate(LocalDate localDate, TimeZone timeZone) { + Objects.requireNonNull(localDate, "localDate must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + long time = ZonedDateTime.of(localDate, LocalTime.MIDNIGHT, timeZone.toZoneId()).toEpochSecond() * 1000; + return new Date(time); + } + + /** + * Converts a {@link LocalTime} to {@link java.sql.Time} using the specified timezone. + * + *For default JVM timezone behavior, use {@link Time#valueOf(LocalTime)} directly.
+ * + * @param localTime the LocalTime to convert + * @param timeZone the timezone context + * @return the java.sql.Time representing the specified time + * @throws NullPointerException if localTime or timeZone is null + */ + public static Time toSqlTime(LocalTime localTime, TimeZone timeZone) { + Objects.requireNonNull(localTime, "localTime must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + ZoneId zoneId = timeZone.toZoneId(); + // java.sql.Time is based on January 1, 1970 + long epochMillis = localTime.atDate(LocalDate.of(1970, 1, 1)) + .atZone(zoneId) + .toInstant() + .toEpochMilli(); + return new Time(epochMillis); + } + + + /** + * Converts a {@link LocalDateTime} to {@link java.sql.Timestamp} using the specified timezone. + * + *Note: This method preserves nanosecond precision from the LocalDateTime.
+ * + *For default JVM timezone behavior, use {@link Timestamp#valueOf(LocalDateTime)} directly.
+ * + * @param localDateTime the LocalDateTime to convert + * @param timeZone the timezone context + * @return the java.sql.Timestamp representing the specified date and time + * @throws NullPointerException if localDateTime or timeZone is null + */ + public static Timestamp toSqlTimestamp(LocalDateTime localDateTime, TimeZone timeZone) { + Objects.requireNonNull(localDateTime, "localDateTime must not be null"); + Objects.requireNonNull(timeZone, "timeZone must not be null"); + + ZoneId zoneId = timeZone.toZoneId(); + Instant instant = localDateTime.atZone(zoneId).toInstant(); + Timestamp timestamp = Timestamp.from(instant); + // Timestamp.from() may lose nanosecond precision, so set it explicitly + timestamp.setNanos(localDateTime.getNano()); + return timestamp; + } } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 2ebf2ffa0..a9b0269c2 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -2,6 +2,7 @@ import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.internal.DataTypeConverter; import com.clickhouse.client.api.internal.MapUtils; @@ -680,13 +681,31 @@ public ZonedDateTime getZonedDateTime(int index) { @Override public Duration getDuration(int index) { - TemporalAmount temporalAmount = getTemporalAmount(index); - return temporalAmount == null ? null : Duration.from(temporalAmount); + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof LocalDateTime) { + return DataTypeUtils.localDateTimeToDuration((LocalDateTime) value); + } else if (value instanceof TemporalAmount) { + return Duration.from((TemporalAmount)value); + } + throw new ClientException("Column at index " + index + " cannot be converted to Duration"); } @Override public TemporalAmount getTemporalAmount(int index) { - return readValue(index); + Object value = readValue(index); + if (value == null) { + return null; + } + if (value instanceof LocalDateTime) { + return DataTypeUtils.localDateTimeToDuration((LocalDateTime) value); + } else if (value instanceof TemporalAmount) { + return (TemporalAmount) value; + } + + throw new ClientException("Column at index " + index + " cannot be converted to TemporalAmount"); } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java index 35d138846..ef5be3c89 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java @@ -1,6 +1,7 @@ package com.clickhouse.client.api.data_formats.internal; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.internal.DataTypeConverter; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.GenericRecord; @@ -173,13 +174,31 @@ public ZonedDateTime getZonedDateTime(String colName) { @Override public Duration getDuration(String colName) { - TemporalAmount temporalAmount = readValue(colName); - return temporalAmount == null ? null : Duration.from(temporalAmount); + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof LocalDateTime) { + return DataTypeUtils.localDateTimeToDuration((LocalDateTime) value); + } else if (value instanceof TemporalAmount) { + return Duration.from((TemporalAmount)value); + } + throw new ClientException("Column " + colName + " cannot be converted to Duration"); } @Override public TemporalAmount getTemporalAmount(String colName) { - return readValue(colName); + Object value = readValue(colName); + if (value == null) { + return null; + } + if (value instanceof LocalDateTime) { + return DataTypeUtils.localDateTimeToDuration((LocalDateTime) value); + } else if (value instanceof TemporalAmount) { + return (TemporalAmount) value; + } + + throw new ClientException("Column " + colName + " cannot be converted to TemporalAmount"); } @Override diff --git a/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java b/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java index cfc3efafd..f7f6126df 100644 --- a/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/api/DataTypeUtilsTests.java @@ -1,19 +1,23 @@ package com.clickhouse.client.api; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import com.clickhouse.data.ClickHouseDataType; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZonedDateTime; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.*; import java.time.temporal.ChronoUnit; +import java.util.Calendar; +import java.util.GregorianCalendar; import java.util.TimeZone; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertThrows; -class DataTypeUtilsTests { + +public class DataTypeUtilsTests { @Test void testDateTimeFormatter() { @@ -130,4 +134,371 @@ void formatInstantForDateTime64Truncated() { "1752980742.232000000"); } + @Test(groups = {"unit"}) + void testDifferentDateConversions() throws Exception { + Calendar externalSystemTz = Calendar.getInstance(TimeZone.getTimeZone("UTC+12")); + Calendar utcTz = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + Calendar applicationLocalTz = Calendar.getInstance(TimeZone.getTimeZone("UTC-8")); + + + String externalDateStr = externalSystemTz.get(Calendar.YEAR) + "-" + (externalSystemTz.get(Calendar.MONTH) + 1) + "-" + externalSystemTz.get(Calendar.DAY_OF_MONTH); + java.sql.Date externalDate = new java.sql.Date(externalSystemTz.getTimeInMillis()); + System.out.println(externalDate.toLocalDate()); + System.out.println(externalDateStr); + System.out.println(externalDate); + + Calendar extCal2 = (Calendar) externalSystemTz.clone(); + extCal2.setTime(externalDate); + + System.out.println("> " + extCal2); + String externalDateStr2 = extCal2.get(Calendar.YEAR) + "-" + (extCal2.get(Calendar.MONTH) + 1) + "-" + extCal2.get(Calendar.DAY_OF_MONTH); + System.out.println("> " + externalDateStr2); + + Calendar extCal3 = (Calendar) externalSystemTz.clone(); + LocalDate localDateFromExternal = externalDate.toLocalDate(); // converted date to local timezone (day may shift) + extCal3.clear(); + extCal3.set(localDateFromExternal.getYear(), localDateFromExternal.getMonthValue() - 1, localDateFromExternal.getDayOfMonth(), 0, 0, 0); + System.out.println("converted> " + extCal3.toInstant()); // wrong date!! + } + @Test(groups = {"unit"}) + void testToLocalDateNullTimeZone() { + Date sqlDate = Date.valueOf("2024-01-15"); + assertThrows(NullPointerException.class, + () -> DataTypeUtils.toLocalDate(sqlDate, (TimeZone) null)); + } + + + @Test(groups = {"unit"}) + void testToLocalDateWithCalendar() { + // Create a date that represents midnight Jan 15, 2024 in UTC + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + utcCal.clear(); + utcCal.set(2024, Calendar.JANUARY, 15, 0, 0, 0); + Date sqlDate = new Date(utcCal.getTimeInMillis()); + + // Using UTC calendar should give us Jan 15 + LocalDate resultUtc = DataTypeUtils.toLocalDate(sqlDate, utcCal.getTimeZone()); + assertEquals(resultUtc, LocalDate.of(2024, 1, 15)); + } + + /** + * Test the "day shift" problem: when a Date's millis are created in one timezone + * but interpreted in another, the day can shift. + */ + @Test(groups = {"unit"}) + void testToLocalDateDayShiftProblem() { + // Simulate: Date created in Pacific/Auckland (UTC+12/+13) + // At midnight Jan 15 in Auckland, it's still Jan 14 in UTC + TimeZone aucklandTz = TimeZone.getTimeZone("Pacific/Auckland"); + Calendar aucklandCal = new GregorianCalendar(aucklandTz); + aucklandCal.clear(); + aucklandCal.set(2024, Calendar.JANUARY, 15, 0, 0, 0); + Date dateFromAuckland = new Date(aucklandCal.getTimeInMillis()); + + // Using Auckland calendar should correctly extract Jan 15 + LocalDate withAucklandCal = DataTypeUtils.toLocalDate(dateFromAuckland, aucklandCal.getTimeZone()); + assertEquals(withAucklandCal, LocalDate.of(2024, 1, 15), + "With correct timezone, should get Jan 15"); + + // Using UTC calendar on the same Date would give a different (earlier) day + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + LocalDate withUtcCal = DataTypeUtils.toLocalDate(dateFromAuckland, utcCal.getTimeZone()); + assertEquals(withUtcCal, LocalDate.of(2024, 1, 14), + "With UTC timezone, should get Jan 14 (day shift demonstrated)"); + } + + @DataProvider(name = "timezonesForDateTest") + public Object[][] timezonesForDateTest() { + return new Object[][] { + {"UTC", "2024-01-15", 2024, 1, 15}, + {"America/New_York", "2024-01-15", 2024, 1, 15}, + {"America/Los_Angeles", "2024-01-15", 2024, 1, 15}, + {"Europe/London", "2024-01-15", 2024, 1, 15}, + {"Europe/Moscow", "2024-01-15", 2024, 1, 15}, + {"Asia/Tokyo", "2024-01-15", 2024, 1, 15}, + {"Pacific/Auckland", "2024-01-15", 2024, 1, 15}, + {"Pacific/Honolulu", "2024-01-15", 2024, 1, 15}, + }; + } + + @Test(groups = {"unit"}, dataProvider = "timezonesForDateTest") + void testToLocalDateWithVariousTimezones(String tzId, String dateStr, int year, int month, int day) { + TimeZone tz = TimeZone.getTimeZone(tzId); + Calendar cal = new GregorianCalendar(tz); + cal.clear(); + cal.set(year, month - 1, day, 0, 0, 0); + Date sqlDate = new Date(cal.getTimeInMillis()); + + LocalDate result = DataTypeUtils.toLocalDate(sqlDate, tz); + assertEquals(result, LocalDate.of(year, month, day), + "Date should be preserved in timezone: " + tzId); + } + + @Test(groups = {"unit"}) + void testToLocalDateWithTimeZoneObject() { + TimeZone utc = TimeZone.getTimeZone("UTC"); + Calendar utcCal = new GregorianCalendar(utc); + utcCal.clear(); + utcCal.set(2024, Calendar.JULY, 4, 0, 0, 0); + Date sqlDate = new Date(utcCal.getTimeInMillis()); + + LocalDate result = DataTypeUtils.toLocalDate(sqlDate, utc); + assertEquals(result, LocalDate.of(2024, 7, 4)); + } + + // ==================== Tests for toLocalTime ==================== + + @Test(groups = {"unit"}) + void testToLocalTimeWithCalendar() { + // Create a time that represents 14:30:00 in UTC + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + utcCal.clear(); + utcCal.set(1970, Calendar.JANUARY, 1, 14, 30, 0); + Time sqlTime = new Time(utcCal.getTimeInMillis()); + + // Using UTC calendar should give us 14:30:00 + LocalTime resultUtc = DataTypeUtils.toLocalTime(sqlTime, utcCal.getTimeZone()); + assertEquals(resultUtc.getHour(), 14); + assertEquals(resultUtc.getMinute(), 30); + assertEquals(resultUtc.getSecond(), 0); + } + + @Test(groups = {"unit"}) + void testToLocalTimeTimeZoneShift() { + // Create time in UTC: 14:00:00 + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + utcCal.clear(); + utcCal.set(1970, Calendar.JANUARY, 1, 14, 0, 0); + Time sqlTime = new Time(utcCal.getTimeInMillis()); + + // In UTC, should be 14:00 + LocalTime inUtc = DataTypeUtils.toLocalTime(sqlTime, utcCal.getTimeZone()); + assertEquals(inUtc, LocalTime.of(14, 0, 0)); + + // In New York (UTC-5), same instant would be 09:00 + Calendar nyCal = new GregorianCalendar(TimeZone.getTimeZone("America/New_York")); + LocalTime inNy = DataTypeUtils.toLocalTime(sqlTime, nyCal.getTimeZone()); + assertEquals(inNy, LocalTime.of(9, 0, 0)); + } + + @Test(groups = {"unit"}) + void testToLocalTimeWithTimeZoneObject() { + TimeZone utc = TimeZone.getTimeZone("UTC"); + Calendar utcCal = new GregorianCalendar(utc); + utcCal.clear(); + utcCal.set(1970, Calendar.JANUARY, 1, 23, 59, 59); + Time sqlTime = new Time(utcCal.getTimeInMillis()); + + LocalTime result = DataTypeUtils.toLocalTime(sqlTime, utc); + assertEquals(result, LocalTime.of(23, 59, 59)); + } + + // ==================== Tests for toLocalDateTime ==================== + + @Test(groups = {"unit"}) + void testToLocalDateTimeTimezoneShift() { + // Create timestamp in UTC: 2024-01-15 04:00:00 + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + utcCal.clear(); + utcCal.set(2024, Calendar.JANUARY, 15, 4, 0, 0); + Timestamp sqlTimestamp = new Timestamp(utcCal.getTimeInMillis()); + + // In UTC: 2024-01-15 04:00:00 + LocalDateTime inUtc = DataTypeUtils.toLocalDateTime(sqlTimestamp, utcCal.getTimeZone()); + assertEquals(inUtc, LocalDateTime.of(2024, 1, 15, 4, 0, 0)); + + // In New York (UTC-5): same instant is 2024-01-14 23:00:00 + Calendar nyCal = new GregorianCalendar(TimeZone.getTimeZone("America/New_York")); + LocalDateTime inNy = DataTypeUtils.toLocalDateTime(sqlTimestamp, nyCal.getTimeZone()); + assertEquals(inNy, LocalDateTime.of(2024, 1, 14, 23, 0, 0)); + } + + @Test(groups = {"unit"}) + void testToLocalDateTimeWithTimeZoneObject() { + TimeZone utc = TimeZone.getTimeZone("UTC"); + Calendar utcCal = new GregorianCalendar(utc); + utcCal.clear(); + utcCal.set(2024, Calendar.DECEMBER, 31, 23, 59, 59); + Timestamp sqlTimestamp = new Timestamp(utcCal.getTimeInMillis()); + sqlTimestamp.setNanos(999999999); + + LocalDateTime result = DataTypeUtils.toLocalDateTime(sqlTimestamp, utc); + assertEquals(result, LocalDateTime.of(2024, 12, 31, 23, 59, 59, 999999999)); + } + + @Test(groups = {"unit"}) + void testToLocalDateTimeNanosPreservedWithTimeZone() { + // Verify nanoseconds are preserved when using TimeZone overload + TimeZone tokyo = TimeZone.getTimeZone("Asia/Tokyo"); + Calendar tokyoCal = new GregorianCalendar(tokyo); + tokyoCal.clear(); + tokyoCal.set(2024, Calendar.JUNE, 15, 10, 30, 45); + Timestamp sqlTimestamp = new Timestamp(tokyoCal.getTimeInMillis()); + sqlTimestamp.setNanos(123456789); + + LocalDateTime result = DataTypeUtils.toLocalDateTime(sqlTimestamp, tokyo); + assertEquals(result.getNano(), 123456789); + assertEquals(result.getHour(), 10); + assertEquals(result.getMinute(), 30); + assertEquals(result.getSecond(), 45); + } + + /** + * Comprehensive test demonstrating the day shift problem and its solution. + */ + @Test(groups = {"unit"}) + void testDayShiftProblemAndSolution() { + // Scenario: Financial system in Tokyo (UTC+9) records a trade at 11 PM on Dec 31 + // Server is running in UTC + TimeZone tokyoTz = TimeZone.getTimeZone("Asia/Tokyo"); + TimeZone utcTz = TimeZone.getTimeZone("UTC"); + + // Trade timestamp: Dec 31, 2024 23:30:00 Tokyo time + Calendar tokyoCal = new GregorianCalendar(tokyoTz); + tokyoCal.clear(); + tokyoCal.set(2024, Calendar.DECEMBER, 31, 23, 30, 0); + Timestamp tradeTimestamp = new Timestamp(tokyoCal.getTimeInMillis()); + + // At 23:30 Tokyo (UTC+9), it's 14:30 UTC - still Dec 31 + LocalDateTime inTokyo = DataTypeUtils.toLocalDateTime(tradeTimestamp, tokyoCal.getTimeZone()); + assertEquals(inTokyo.toLocalDate(), LocalDate.of(2024, 12, 31), + "In Tokyo timezone, trade date should be Dec 31"); + + LocalDateTime inUtc = DataTypeUtils.toLocalDateTime(tradeTimestamp, + new GregorianCalendar(utcTz).getTimeZone()); + assertEquals(inUtc.toLocalDate(), LocalDate.of(2024, 12, 31), + "In UTC, same trade is also Dec 31 (14:30 UTC)"); + + // But if the trade was at 00:30 Tokyo time on Jan 1... + tokyoCal.clear(); + tokyoCal.set(2025, Calendar.JANUARY, 1, 0, 30, 0); + Timestamp newYearTrade = new Timestamp(tokyoCal.getTimeInMillis()); + + LocalDateTime newYearInTokyo = DataTypeUtils.toLocalDateTime(newYearTrade, tokyoCal.getTimeZone()); + assertEquals(newYearInTokyo.toLocalDate(), LocalDate.of(2025, 1, 1), + "In Tokyo, it's New Year's Day"); + + LocalDateTime newYearInUtc = DataTypeUtils.toLocalDateTime(newYearTrade, + new GregorianCalendar(utcTz).getTimeZone()); + assertEquals(newYearInUtc.toLocalDate(), LocalDate.of(2024, 12, 31), + "In UTC, it's still Dec 31 (15:30 UTC on Dec 31)"); + } + + // ==================== Tests for toSqlDate ==================== + + @Test(groups = {"unit"}) + void testToSqlDateNullTimeZone() { + LocalDate localDate = LocalDate.of(2024, 1, 15); + assertThrows(NullPointerException.class, + () -> DataTypeUtils.toSqlDate(localDate, (TimeZone) null)); + } + + @Test(groups = {"unit"}) + void testToSqlDateWithTimeZone() { + LocalDate localDate = LocalDate.of(2024, 7, 4); + TimeZone utc = TimeZone.getTimeZone("UTC"); + + Date sqlDate = DataTypeUtils.toSqlDate(localDate, utc); + + // Convert back to verify round-trip + LocalDate roundTrip = DataTypeUtils.toLocalDate(sqlDate, utc); + assertEquals(roundTrip, localDate); + } + + @Test(groups = {"unit"}) + void testToSqlDateRoundTripWithVariousTimezones() { + LocalDate localDate = LocalDate.of(2024, 1, 15); + String[] tzIds = {"UTC", "America/New_York", "Asia/Tokyo", "Pacific/Auckland"}; + + for (String tzId : tzIds) { + TimeZone tz = TimeZone.getTimeZone(tzId); + Calendar cal = new GregorianCalendar(tz); + + // Convert to SQL Date and back + Date sqlDate = DataTypeUtils.toSqlDate(localDate, cal.getTimeZone()); + LocalDate roundTrip = DataTypeUtils.toLocalDate(sqlDate, cal.getTimeZone()); + + assertEquals(roundTrip, localDate, + "Round-trip should preserve date in timezone: " + tzId); + } + } + + // ==================== Tests for toSqlTimestamp ==================== + + @Test(groups = {"unit"}) + void testToSqlTimestampWithTimeZone() { + LocalDateTime localDateTime = LocalDateTime.of(2024, 12, 31, 23, 59, 59, 999999999); + TimeZone utc = TimeZone.getTimeZone("UTC"); + + Timestamp sqlTimestamp = DataTypeUtils.toSqlTimestamp(localDateTime, utc); + + // Convert back to verify round-trip + LocalDateTime roundTrip = DataTypeUtils.toLocalDateTime(sqlTimestamp, utc); + assertEquals(roundTrip, localDateTime); + } + + @Test(groups = {"unit"}) + void testToSqlTimestampPreservesNanoseconds() { + LocalDateTime localDateTime = LocalDateTime.of(2024, 6, 15, 10, 30, 45, 123456789); + Calendar utcCal = new GregorianCalendar(TimeZone.getTimeZone("UTC")); + + Timestamp sqlTimestamp = DataTypeUtils.toSqlTimestamp(localDateTime, utcCal.getTimeZone()); + + assertEquals(sqlTimestamp.getNanos(), 123456789); + } + + @Test(groups = {"unit"}) + void testToSqlTimestampRoundTripWithVariousTimezones() { + LocalDateTime localDateTime = LocalDateTime.of(2024, 1, 15, 23, 30, 45, 123456789); + String[] tzIds = {"UTC", "America/New_York", "Asia/Tokyo", "Pacific/Auckland"}; + + for (String tzId : tzIds) { + TimeZone tz = TimeZone.getTimeZone(tzId); + Calendar cal = new GregorianCalendar(tz); + + // Convert to SQL Timestamp and back + Timestamp sqlTimestamp = DataTypeUtils.toSqlTimestamp(localDateTime, cal.getTimeZone()); + LocalDateTime roundTrip = DataTypeUtils.toLocalDateTime(sqlTimestamp, cal.getTimeZone()); + + assertEquals(roundTrip, localDateTime, + "Round-trip should preserve datetime in timezone: " + tzId); + } + } + + /** + * Comprehensive round-trip test demonstrating timezone handling. + */ + @Test(groups = {"unit"}) + void testRoundTripConversionsWithDifferentTimezones() { + // Original values + LocalDate date = LocalDate.of(2024, 7, 4); + LocalTime time = LocalTime.of(14, 30, 45, 123000000); + LocalDateTime dateTime = LocalDateTime.of(date, time); + + TimeZone tokyo = TimeZone.getTimeZone("Asia/Tokyo"); + TimeZone newYork = TimeZone.getTimeZone("America/New_York"); + + // Convert to SQL types using Tokyo timezone + Calendar tokyoCal = new GregorianCalendar(tokyo); + Date sqlDateTokyo = DataTypeUtils.toSqlDate(date, tokyoCal.getTimeZone()); + Time sqlTimeTokyo = DataTypeUtils.toSqlTime(time, tokyoCal.getTimeZone()); + Timestamp sqlTimestampTokyo = DataTypeUtils.toSqlTimestamp(dateTime, tokyoCal.getTimeZone()); + + // Round-trip back using same timezone should preserve values + assertEquals(DataTypeUtils.toLocalDate(sqlDateTokyo, tokyoCal.getTimeZone()), date); + LocalTime timeRoundTrip = DataTypeUtils.toLocalTime(sqlTimeTokyo, tokyoCal.getTimeZone()); + assertEquals(timeRoundTrip.getHour(), time.getHour()); + assertEquals(timeRoundTrip.getMinute(), time.getMinute()); + assertEquals(timeRoundTrip.getSecond(), time.getSecond()); + assertEquals(DataTypeUtils.toLocalDateTime(sqlTimestampTokyo, tokyoCal.getTimeZone()), dateTime); + + // If we interpret the same SQL values in a different timezone, we get different local values + // This is expected - the same instant in time represents different local times in different zones + Calendar nyCal = new GregorianCalendar(newYork); + LocalDateTime dateTimeInNy = DataTypeUtils.toLocalDateTime(sqlTimestampTokyo, nyCal.getTimeZone()); + // Tokyo is 13-14 hours ahead of NY, so the local time should be different + // (14:30 Tokyo = 01:30 or 00:30 NY depending on DST) + assertEquals(dateTimeInNy.toLocalDate(), LocalDate.of(2024, 7, 4).minusDays(1), + "Same instant should be previous day in New York"); + } } diff --git a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java index 52347c729..59ebd1467 100644 --- a/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java @@ -32,6 +32,8 @@ import java.lang.reflect.Method; import java.math.BigDecimal; import java.math.RoundingMode; +import java.sql.Connection; +import java.time.Duration; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -860,22 +862,26 @@ public void testTimeDataType() throws Exception { Assert.assertEquals(record.getInteger("o_num"), 1); Assert.assertEquals(record.getLocalDateTime("time").toEpochSecond(ZoneOffset.UTC), TimeUnit.HOURS.toSeconds(999)); Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999))); + Assert.assertEquals(record.getDuration("time"), Duration.ofHours(999)); record = records.get(1); Assert.assertEquals(record.getInteger("o_num"), 2); Assert.assertEquals(record.getLocalDateTime("time").toEpochSecond(ZoneOffset.UTC), TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59); Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); + Assert.assertEquals(record.getDuration("time"), Duration.ofHours(999).plusMinutes(59).plusSeconds(59)); record = records.get(2); Assert.assertEquals(record.getInteger("o_num"), 3); Assert.assertEquals(record.getLocalDateTime("time").toEpochSecond(ZoneOffset.UTC), 0); Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(0)); + Assert.assertEquals(record.getDuration("time"), Duration.ofHours(0)); record = records.get(3); Assert.assertEquals(record.getInteger("o_num"), 4); Assert.assertEquals(record.getLocalDateTime("time").toEpochSecond(ZoneOffset.UTC), - (TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59)); Assert.assertEquals(record.getInstant("time"), Instant.ofEpochSecond(- (TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59))); + Assert.assertEquals(record.getDuration("time"), Duration.ofHours(999).plusMinutes(59).plusSeconds(59).negated()); } @Test(groups = {"integration"}, dataProvider = "testTimeData") @@ -1138,7 +1144,7 @@ public void testDates() throws Exception { } @Test(groups = {"integration"}, dataProvider = "testNestedArrays_dp") - public void testNestedArrays(String columnDef, String insertValues, String[] expectedStrValues, + public void testNestedArrays(String columnDef, String insertValues, String[] expectedStrValues, String[] expectedListValues) throws Exception { final String table = "test_nested_arrays"; client.execute("DROP TABLE IF EXISTS " + table).get(); @@ -1151,12 +1157,12 @@ public void testNestedArrays(String columnDef, String insertValues, String[] exp for (GenericRecord record : records) { int rowId = record.getInteger("rowId"); - + // Check getString() - includes quotes for string values String actualValue = record.getString("arr"); - Assert.assertEquals(actualValue, expectedStrValues[rowId - 1], + Assert.assertEquals(actualValue, expectedStrValues[rowId - 1], "getString() mismatch at row " + rowId + " for " + columnDef); - + // Check getObject() - should return an ArrayValue Object objValue = record.getObject("arr"); Assert.assertNotNull(objValue, "getObject() returned null at row " + rowId); @@ -1165,7 +1171,7 @@ public void testNestedArrays(String columnDef, String insertValues, String[] exp BinaryStreamReader.ArrayValue arrayValue = (BinaryStreamReader.ArrayValue) objValue; Assert.assertEquals(arrayValue.asList().toString(), expectedListValues[rowId - 1], "getObject().asList() mismatch at row " + rowId + " for " + columnDef); - + // Check getList() - should return a List representation (no quotes for strings) List> listValue = record.getList("arr"); Assert.assertNotNull(listValue, "getList() returned null at row " + rowId); diff --git a/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java b/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java index 57ef3cb5c..e0d84a17e 100644 --- a/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/internal/SmallTests.java @@ -2,11 +2,14 @@ import org.testng.annotations.Test; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.TimeZone; import java.util.concurrent.TimeUnit; /** @@ -46,4 +49,35 @@ public void testInstantVsLocalTime() { System.out.println(maxTime); System.out.println(maxTime.getDayOfYear()); } + + @Test + public void testInstantFromUTC() { + + LocalDate ld = LocalDate.of(1970, 1, 1); + + ZonedDateTime atTokyo = ld.atStartOfDay(TimeZone.getTimeZone("Asia/Tokyo").toZoneId()); + Instant tokyoInstant = atTokyo.toInstant(); + + ZonedDateTime atUtc = ld.atStartOfDay(TimeZone.getTimeZone("UTC").toZoneId()); + Instant utcInstant = atUtc.toInstant(); + + System.out.println(ld); + System.out.println(atTokyo); + System.out.println(tokyoInstant); + System.out.println(atUtc); + System.out.println(utcInstant); + + } + + @Test + public void testTimezoneOffset() { + ZoneId tokyoTz = ZoneId.of("Asia/Tokyo"); + ZoneId losAngelesTz = ZoneId.of("America/Los_Angeles"); + + System.out.println(tokyoTz.getRules().getTransitionRules()); + System.out.println(losAngelesTz.getRules().getTransitionRules()); + + ZonedDateTime ld = LocalDate.of(1970, 3, 7).atStartOfDay(losAngelesTz); + System.out.println(ld.toOffsetDateTime()); + } } diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java index fcd17b6d6..9bdce577c 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/PreparedStatementImpl.java @@ -42,16 +42,14 @@ import java.sql.Struct; import java.sql.Time; import java.sql.Timestamp; +import java.sql.Types; +import java.time.Duration; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; -import java.time.ZoneId; import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeFormatterBuilder; -import java.time.temporal.ChronoField; import java.util.ArrayList; import java.util.Arrays; import java.util.Calendar; @@ -69,11 +67,6 @@ public class PreparedStatementImpl extends StatementImpl implements PreparedStatement, JdbcV2Wrapper { private static final Logger LOG = LoggerFactory.getLogger(PreparedStatementImpl.class); - public static final DateTimeFormatter TIME_FORMATTER = new DateTimeFormatterBuilder().appendPattern("HH:mm:ss") - .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true).toFormatter(); - public static final DateTimeFormatter DATETIME_FORMATTER = new DateTimeFormatterBuilder() - .appendPattern("yyyy-MM-dd HH:mm:ss").appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true).toFormatter(); - private final Calendar defaultCalendar; private final String originalSql; @@ -218,17 +211,17 @@ public void setBytes(int parameterIndex, byte[] x) throws SQLException { @Override public void setDate(int parameterIndex, Date x) throws SQLException { - setDate(parameterIndex, x, null); + setDate(parameterIndex, x, defaultCalendar); } @Override public void setTime(int parameterIndex, Time x) throws SQLException { - setTime(parameterIndex, x, null); + setTime(parameterIndex, x, defaultCalendar); } @Override public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { - setTimestamp(parameterIndex, x, null); + setTimestamp(parameterIndex, x, defaultCalendar); } @Override @@ -262,27 +255,31 @@ int getParametersCount() { @Override public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { ensureOpen(); - // targetSQLType is only of JDBCType - values[parameterIndex-1] = encodeObject(x, jdbcType2ClickHouseDataType(JDBCType.valueOf(targetSqlType)), null); + + isValidForTargetType(x, targetSqlType); + values[parameterIndex-1] = encodeObject(x); } @Override public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException { ensureOpen(); - // targetSQLType is only of JDBCType - values[parameterIndex-1] = encodeObject(x, jdbcType2ClickHouseDataType(JDBCType.valueOf(targetSqlType)), scaleOrLength); + + isValidForTargetType(x, targetSqlType); + values[parameterIndex-1] = encodeObject(x, (long) scaleOrLength); } @Override public void setObject(int parameterIndex, Object x, SQLType targetSqlType) throws SQLException { ensureOpen(); - values[parameterIndex-1] = encodeObject(x, sqlType2ClickHouseDataType(targetSqlType), null); + + isValidForTargetType(x, targetSqlType.getVendorTypeNumber()); + values[parameterIndex-1] = encodeObject(x); } @Override public void setObject(int parameterIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException { ensureOpen(); - values[parameterIndex-1] = encodeObject(x, sqlType2ClickHouseDataType(targetSqlType), scaleOrLength); + values[parameterIndex-1] = encodeObject(x, (long) scaleOrLength); } @Override @@ -469,43 +466,19 @@ public static String replaceQuestionMarks(String sql, final String replacement) @Override public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { ensureOpen(); - values[parameterIndex - 1] = encodeObject(sqlDateToInstant(x, cal)); - } - - protected Instant sqlDateToInstant(Date x, Calendar cal) { - LocalDate d = x.toLocalDate(); - Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); - c.clear(); - c.set(d.getYear(), d.getMonthValue() - 1, d.getDayOfMonth(), 0, 0, 0); - return c.toInstant(); + values[parameterIndex - 1] = encodeObject(DataTypeUtils.toLocalDate(x, cal.getTimeZone())); } @Override public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { ensureOpen(); - values[parameterIndex - 1] = encodeObject(sqlTimeToInstant(x, cal)); - } - - protected Instant sqlTimeToInstant(Time x, Calendar cal) { - LocalTime t = x.toLocalTime(); - Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); - c.clear(); - c.set(1970, Calendar.JANUARY, 1, t.getHour(), t.getMinute(), t.getSecond()); - return c.toInstant(); + values[parameterIndex - 1] = encodeObject(DataTypeUtils.toLocalTime(x, cal.getTimeZone())); } @Override public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { ensureOpen(); - values[parameterIndex - 1] = encodeObject(sqlTimestampToZDT(x, cal)); - } - - protected ZonedDateTime sqlTimestampToZDT(Timestamp x, Calendar cal) { - LocalDateTime ldt = x.toLocalDateTime(); - Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); - c.clear(); - c.set(ldt.getYear(), ldt.getMonthValue() - 1, ldt.getDayOfMonth(), ldt.getHour(), ldt.getMinute(), ldt.getSecond()); - return c.toInstant().atZone(ZoneId.of("UTC")).withNano(x.getNanos()); + values[parameterIndex - 1] = encodeObject(DataTypeUtils.toLocalDateTime(x, cal.getTimeZone())); } @Override @@ -783,19 +756,21 @@ private String encodeObject(Object x, Long length) throws SQLException { } else if (x instanceof LocalDate) { return QUOTE + DataTypeUtils.DATE_FORMATTER.format((LocalDate) x) + QUOTE; } else if (x instanceof Time) { - return QUOTE + TIME_FORMATTER.format(((Time) x).toLocalTime()) + QUOTE; + return QUOTE + DataTypeUtils.TIME_FORMATTER.format(((Time) x).toLocalTime()) + QUOTE; } else if (x instanceof LocalTime) { - return QUOTE + TIME_FORMATTER.format((LocalTime) x) + QUOTE; + return QUOTE + DataTypeUtils.TIME_FORMATTER.format((LocalTime) x) + QUOTE; } else if (x instanceof Timestamp) { - return QUOTE + DATETIME_FORMATTER.format(((Timestamp) x).toLocalDateTime()) + QUOTE; + return QUOTE + DataTypeUtils.DATE_TIME_WITH_OPTIONAL_NANOS.format(((Timestamp) x).toLocalDateTime()) + QUOTE; } else if (x instanceof LocalDateTime) { - return QUOTE + DATETIME_FORMATTER.format((LocalDateTime) x) + QUOTE; + return QUOTE + DataTypeUtils.DATE_TIME_WITH_OPTIONAL_NANOS.format((LocalDateTime) x) + QUOTE; } else if (x instanceof OffsetDateTime) { return encodeObject(((OffsetDateTime) x).toInstant()); } else if (x instanceof ZonedDateTime) { return encodeObject(((ZonedDateTime) x).toInstant()); } else if (x instanceof Instant) { return "fromUnixTimestamp64Nano(" + (((Instant) x).getEpochSecond() * 1_000_000_000L + ((Instant) x).getNano()) + ")"; + } else if (x instanceof Duration) { + return QUOTE + DataTypeUtils.durationToTimeString((Duration) x, 9) + QUOTE; } else if (x instanceof InetAddress) { return QUOTE + ((InetAddress) x).getHostAddress() + QUOTE; } else if (x instanceof byte[]) { @@ -1006,45 +981,36 @@ private static String encodeCharacterStream(Reader reader, Long length) throws S } } - private ClickHouseDataType jdbcType2ClickHouseDataType(JDBCType type) throws SQLException{ - ClickHouseDataType clickHouseDataType = JdbcUtils.SQL_TO_CLICKHOUSE_TYPE_MAP.get(type); - if (clickHouseDataType == null) { - throw new SQLException("Cannot convert " + type + " to a ClickHouse one. Consider using java.sql.JDBCType or com.clickhouse.data.ClickHouseDataType"); - } - - return clickHouseDataType; - } - - private ClickHouseDataType sqlType2ClickHouseDataType(SQLType type) throws SQLException { - ClickHouseDataType clickHouseDataType = null; - if (type instanceof JDBCType) { - clickHouseDataType = JdbcUtils.SQL_TO_CLICKHOUSE_TYPE_MAP.get(type); - } else if (type instanceof ClickHouseDataType) { - clickHouseDataType = (ClickHouseDataType) type; - if (JdbcUtils.INVALID_TARGET_TYPES.contains(clickHouseDataType)) { - throw new SQLException("Type " + clickHouseDataType + " cannot be used as target type here because requires additional parameters and API doesn't have a way to pass them. "); - } + private void isValidForTargetType(Object value, int targetType) throws SQLException { + if (value == null) { + return; // NULL is handled in encoding and server checks if value can be NULL } - if (clickHouseDataType == null) { - throw new SQLException("Cannot convert " + type + " to a ClickHouse one. Consider using java.sql.JDBCType or com.clickhouse.data.ClickHouseDataType"); - } + Class> vClass = value.getClass(); - return clickHouseDataType; - } - - private String encodeObject(Object x, ClickHouseDataType clickHouseDataType, Integer scaleOrLength) throws SQLException { - String encodedObject = encodeObject(x); - if (clickHouseDataType != null) { - encodedObject = "CAST (" + encodedObject + " AS " + clickHouseDataType.name(); - if (clickHouseDataType.hasParameter()) { - if (scaleOrLength == null) { - throw new SQLException("Target type " + clickHouseDataType + " requires a parameter"); + // Here we validate only specific types + switch (targetType) { + case Types.DATE: + if (vClass == LocalDate.class || vClass == java.sql.Date.class) { + return; } - encodedObject += "(" + scaleOrLength + ")"; - } - encodedObject += ")"; + break; + case Types.TIME: + case Types.TIME_WITH_TIMEZONE: + if (vClass == LocalTime.class || vClass == java.sql.Time.class || vClass == LocalDateTime.class || vClass == ZonedDateTime.class) { + return; + } + break; + case Types.TIMESTAMP: + case Types.TIMESTAMP_WITH_TIMEZONE: + if (vClass == Timestamp.class || vClass == LocalDateTime.class || vClass == ZonedDateTime.class) { + return; + } + break; + default: + return; } - return encodedObject; + + throw new SQLException("Cannot convert value of type " + value.getClass() + " to SQL type " + JDBCType.valueOf(targetType)); } } diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java index 2f8d383e3..4d8454e31 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java @@ -1,5 +1,6 @@ package com.clickhouse.jdbc; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.metadata.TableSchema; import com.clickhouse.client.api.query.QueryResponse; @@ -33,8 +34,13 @@ import java.sql.Statement; import java.sql.Time; import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Calendar; import java.util.Collections; @@ -244,17 +250,17 @@ public byte[] getBytes(int columnIndex) throws SQLException { @Override public Date getDate(int columnIndex) throws SQLException { - return getDate(columnIndex, null); + return getDate(columnIndex, defaultCalendar); } @Override public Time getTime(int columnIndex) throws SQLException { - return getTime(columnIndex, null); + return getTime(columnIndex, defaultCalendar); } @Override public Timestamp getTimestamp(int columnIndex) throws SQLException { - return getTimestamp(columnIndex, null); + return getTimestamp(columnIndex, defaultCalendar); } @Override @@ -423,17 +429,17 @@ public byte[] getBytes(String columnLabel) throws SQLException { @Override public Date getDate(String columnLabel) throws SQLException { - return getDate(columnLabel, null); + return getDate(columnLabel, defaultCalendar); } @Override public Time getTime(String columnLabel) throws SQLException { - return getTime(columnLabel, null); + return getTime(columnLabel, defaultCalendar); } @Override public Timestamp getTimestamp(String columnLabel) throws SQLException { - return getTimestamp(columnLabel, null); + return getTimestamp(columnLabel, defaultCalendar); } @Override @@ -1015,6 +1021,7 @@ public Date getDate(int columnIndex, Calendar cal) throws SQLException { public Date getDate(String columnLabel, Calendar cal) throws SQLException { checkClosed(); try { + LocalDate ld = reader.getLocalDate(columnLabel); if (ld == null) { wasNull = true; @@ -1022,10 +1029,7 @@ public Date getDate(String columnLabel, Calendar cal) throws SQLException { } wasNull = false; - Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); - c.clear(); - c.set(ld.getYear(), ld.getMonthValue() - 1, ld.getDayOfMonth(), 0, 0, 0); - return new Date(c.getTimeInMillis()); + return DataTypeUtils.toSqlDate(ld, cal.getTimeZone()); } catch (Exception e) { ClickHouseColumn column = getSchema().getColumnByName(columnLabel); switch (column.getValueDataType()) { @@ -1080,28 +1084,18 @@ public Time getTime(String columnLabel, Calendar cal) throws SQLException { @Override public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException { - return getTimestamp(columnIndexToName(columnIndex), cal); - } - - @Override - public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException { checkClosed(); try { - ZonedDateTime zdt = reader.getZonedDateTime(columnLabel); + ZonedDateTime zdt = reader.getZonedDateTime(columnIndex); if (zdt == null) { wasNull = true; return null; } wasNull = false; - Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone(); - c.set(zdt.getYear(), zdt.getMonthValue() - 1, zdt.getDayOfMonth(), zdt.getHour(), zdt.getMinute(), - zdt.getSecond()); - Timestamp timestamp = new Timestamp(c.getTimeInMillis()); - timestamp.setNanos(zdt.getNano()); - return timestamp; + return DataTypeUtils.toSqlTimestamp(zdt.toLocalDateTime(), cal.getTimeZone()); } catch (Exception e) { - ClickHouseColumn column = getSchema().getColumnByName(columnLabel); + ClickHouseColumn column = getSchema().getColumnByIndex(columnIndex); switch (column.getValueDataType()) { case DateTime64: case DateTime: @@ -1111,8 +1105,15 @@ public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLExcept throw new SQLException("Value of " + column.getValueDataType() + " type cannot be converted to Timestamp value"); } - throw ExceptionUtils.toSqlState(String.format("Method: getTimestamp(\"%s\") encountered an exception.", columnLabel), String.format("SQL: [%s]", parentStatement.getLastStatementSql()), e); + throw ExceptionUtils.toSqlState(String.format("Method: getTimestamp(\"%s\") encountered an exception.", columnIndex), + String.format("SQL: [%s]", parentStatement.getLastStatementSql()), e); } + + } + + @Override + public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException { + return getTimestamp(getSchema().nameToColumnIndex(columnLabel), cal); } @Override diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/WriterStatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/WriterStatementImpl.java index c437c5731..18bc4564d 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/WriterStatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/WriterStatementImpl.java @@ -1,5 +1,6 @@ package com.clickhouse.jdbc; +import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatWriter; import com.clickhouse.client.api.data_formats.RowBinaryFormatWriter; import com.clickhouse.client.api.insert.InsertResponse; @@ -47,12 +48,14 @@ public class WriterStatementImpl extends PreparedStatementImpl implements Prepar private ByteArrayOutputStream out; private ClickHouseBinaryFormatWriter writer; private final TableSchema tableSchema; + private final Calendar defaultCalendar; public WriterStatementImpl(ConnectionImpl connection, String originalSql, TableSchema tableSchema, ParsedPreparedStatement parsedStatement) throws SQLException { super(connection, originalSql, parsedStatement); + this.defaultCalendar = connection.getDefaultCalendar(); if (parsedStatement.getInsertColumns() != null) { List