Skip to content

Commit

Permalink
Merge pull request #2107 from ClickHouse/timezone-testing
Browse files Browse the repository at this point in the history
Timezone testing
  • Loading branch information
Paultagoras authored Jan 23, 2025
2 parents 2581f3b + 3f57bc5 commit 4807241
Show file tree
Hide file tree
Showing 7 changed files with 237 additions and 78 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
package com.clickhouse.jdbc.comparison;

import com.clickhouse.client.ClickHouseProtocol;
import com.clickhouse.client.ClickHouseServerForTest;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.jdbc.ConnectionImpl;
import com.clickhouse.jdbc.JdbcIntegrationTest;
import com.clickhouse.jdbc.internal.ClickHouseConnectionImpl;
import org.testng.Assert;
import org.testng.annotations.Test;

import java.sql.Connection;
import java.sql.Date;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.sql.Timestamp;
import java.util.GregorianCalendar;
import java.util.Properties;
import java.util.TimeZone;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertThrows;
import static org.testng.Assert.assertTrue;

public class DateTimeComparisonTest extends JdbcIntegrationTest {
public Connection getJdbcConnectionV1(Properties properties) throws SQLException {
Properties info = new Properties();
info.setProperty("user", "default");
info.setProperty("password", ClickHouseServerForTest.getPassword());
info.setProperty("database", ClickHouseServerForTest.getDatabase());

if (properties != null) {
info.putAll(properties);
}

return new ClickHouseConnectionImpl(getJDBCEndpointString(), info);
}

public Connection getJdbcConnectionV2(Properties properties) throws SQLException {
Properties info = new Properties();
info.setProperty("user", "default");
info.setProperty("password", ClickHouseServerForTest.getPassword());
info.setProperty("database", ClickHouseServerForTest.getDatabase());

if (properties != null) {
info.putAll(properties);
}

info.setProperty(ClientConfigProperties.DATABASE.getKey(), ClickHouseServerForTest.getDatabase());

return new ConnectionImpl(getJDBCEndpointString(), info);
}

public String getJDBCEndpointString() {
return "jdbc:ch:" + (isCloud() ? "" : "http://") +
ClickHouseServerForTest.getClickHouseAddress(ClickHouseProtocol.HTTP, false) + "/" + (isCloud() ? ClickHouseServerForTest.getDatabase() : "");
}

private void run(String query) throws SQLException {
try (Connection connection = getJdbcConnectionV2(null)) {
try (Statement stmt = connection.createStatement()) {
stmt.execute("CREATE DATABASE IF NOT EXISTS " + ClickHouseServerForTest.getDatabase());
stmt.execute(query);
}
}
}

@Test (groups = "integration", enabled = true)
public void setDateTest() throws SQLException {
run("DROP TABLE IF EXISTS test_date");
run("CREATE TABLE IF NOT EXISTS test_date (id Int8, d1 Date, d2 Date, d3 Date) ENGINE = MergeTree ORDER BY id");

try (Connection connV1 = getJdbcConnectionV1(null)) {
try (PreparedStatement stmtV1 = connV1.prepareStatement("INSERT INTO test_date VALUES (1, ?, ?, ?)")) {//INSERT with V1
stmtV1.setDate(1, java.sql.Date.valueOf("2021-01-01"));
stmtV1.setDate(2, java.sql.Date.valueOf("2021-01-01"), new GregorianCalendar());
stmtV1.setDate(3, java.sql.Date.valueOf("2021-01-01"), new GregorianCalendar(TimeZone.getTimeZone("UTC")));
stmtV1.execute();
}
}

try (Connection connV1 = getJdbcConnectionV1(null);
Connection connV2 = getJdbcConnectionV2(null)) {
try (Statement stmtV1 = connV1.createStatement();
Statement stmtV2 = connV2.createStatement()) {
ResultSet rsV1 = stmtV1.executeQuery("SELECT * FROM test_date");
ResultSet rsV2 = stmtV2.executeQuery("SELECT * FROM test_date");
assertTrue(rsV1.next());
assertTrue(rsV2.next());

assertEquals(rsV2.getDate(2), rsV1.getDate(2));
assertEquals(rsV2.getDate(3, new GregorianCalendar()), rsV1.getDate(3, new GregorianCalendar()));
assertEquals(rsV2.getDate(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))),
rsV1.getDate(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))));
}
}
}


@Test (groups = "integration", enabled = true)
public void setTimeTest() throws SQLException {
run("DROP TABLE IF EXISTS test_time");
run("CREATE TABLE IF NOT EXISTS test_time (id Int8, t1 Datetime, t2 Datetime, t3 Datetime) ENGINE = MergeTree ORDER BY id");

try (Connection connV1 = getJdbcConnectionV1(null)) {
try (PreparedStatement stmtV1 = connV1.prepareStatement("INSERT INTO test_time VALUES (1, ?, ?, ?)")) {//INSERT with V1
stmtV1.setTime(1, java.sql.Time.valueOf("12:34:56"));
stmtV1.setTime(2, java.sql.Time.valueOf("12:34:56"), new GregorianCalendar());
stmtV1.setTime(3, java.sql.Time.valueOf("12:34:56"), new GregorianCalendar(TimeZone.getTimeZone("UTC")));
stmtV1.execute();
}
}

try (Connection connV1 = getJdbcConnectionV1(null);
Connection connV2 = getJdbcConnectionV2(null)) {
try (Statement stmtV1 = connV1.createStatement();
Statement stmtV2 = connV2.createStatement()) {
ResultSet rsV1 = stmtV1.executeQuery("SELECT * FROM test_time");
ResultSet rsV2 = stmtV2.executeQuery("SELECT * FROM test_time");
assertTrue(rsV1.next());
assertTrue(rsV2.next());

assertEquals(rsV2.getTime(2), rsV1.getTime(2));
assertEquals(rsV2.getTime(3, new GregorianCalendar()), rsV1.getTime(3, new GregorianCalendar()));
assertEquals(rsV2.getTime(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))),
rsV1.getTime(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))));
}
}
}

@Test (groups = "integration", enabled = true)
public void setTimestampTest() throws SQLException {
run("DROP TABLE IF EXISTS test_timestamp");
run("CREATE TABLE IF NOT EXISTS test_timestamp (id Int8, t1 Datetime64(3), t2 Datetime64(6), t3 Datetime64(9), t4 DateTime64(9)) ENGINE = MergeTree ORDER BY id");

Timestamp ts = new Timestamp(System.currentTimeMillis());

try (Connection connV1 = getJdbcConnectionV1(null)) {
try (PreparedStatement stmtV1 = connV1.prepareStatement("INSERT INTO test_timestamp VALUES (1, ?, ?, ?, ?)")) {//INSERT with V1
stmtV1.setTimestamp(1, java.sql.Timestamp.valueOf("2021-01-01 01:23:45"));
stmtV1.setTimestamp(2, java.sql.Timestamp.valueOf("2021-01-01 01:23:45"), new GregorianCalendar());
stmtV1.setTimestamp(3, java.sql.Timestamp.valueOf("2021-01-01 01:23:45"), new GregorianCalendar(TimeZone.getTimeZone("UTC")));
stmtV1.setTimestamp(4, ts);
stmtV1.execute();
}
}

try (Connection connV1 = getJdbcConnectionV1(null);
Connection connV2 = getJdbcConnectionV2(null)) {
try (Statement stmtV1 = connV1.createStatement();
Statement stmtV2 = connV2.createStatement()) {
ResultSet rsV1 = stmtV1.executeQuery("SELECT * FROM test_timestamp");
ResultSet rsV2 = stmtV2.executeQuery("SELECT * FROM test_timestamp");
assertTrue(rsV1.next());
assertTrue(rsV2.next());

assertEquals(rsV2.getTimestamp(2), rsV1.getTimestamp(2));
assertEquals(rsV2.getTimestamp(3, new GregorianCalendar()), rsV1.getTimestamp(3, new GregorianCalendar()));
assertEquals(rsV2.getTimestamp(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))),
rsV1.getTimestamp(4, new GregorianCalendar(TimeZone.getTimeZone("UTC"))));
assertEquals(rsV2.getTimestamp(5), rsV1.getTimestamp(5));
}
}
}
}
3 changes: 3 additions & 0 deletions jdbc-v2/src/main/java/com/clickhouse/jdbc/ConnectionImpl.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.sql.ShardingKey;
import java.sql.Statement;
import java.sql.Struct;
import java.util.Calendar;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
Expand All @@ -52,6 +53,7 @@ public class ConnectionImpl implements Connection, JdbcV2Wrapper {
private QuerySettings defaultQuerySettings;

private final com.clickhouse.jdbc.metadata.DatabaseMetaData metadata;
protected final Calendar defaultCalendar;

public ConnectionImpl(String url, Properties info) throws SQLException {
try {
Expand Down Expand Up @@ -80,6 +82,7 @@ public ConnectionImpl(String url, Properties info) throws SQLException {
.serverSetting(ServerSettings.WAIT_END_OF_QUERY, "0");

this.metadata = new com.clickhouse.jdbc.metadata.DatabaseMetaData(this, false, url);
this.defaultCalendar = Calendar.getInstance();
} catch (SQLException e) {
throw e;
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
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;
Expand All @@ -41,7 +42,6 @@
import java.util.Collection;
import java.util.GregorianCalendar;
import java.util.Map;
import java.util.TimeZone;

public class PreparedStatementImpl extends StatementImpl implements PreparedStatement, JdbcV2Wrapper {
private static final Logger LOG = LoggerFactory.getLogger(PreparedStatementImpl.class);
Expand All @@ -52,6 +52,8 @@ public class PreparedStatementImpl extends StatementImpl implements PreparedStat
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;

String originalSql;
String [] sqlSegments;
Object [] parameters;
Expand All @@ -68,6 +70,8 @@ public PreparedStatementImpl(ConnectionImpl connection, String sql) throws SQLEx
} else {
this.parameters = new Object[0];
}

this.defaultCalendar = connection.defaultCalendar;
}

private String compileSql() {
Expand Down Expand Up @@ -268,12 +272,8 @@ public ResultSetMetaData getMetaData() throws SQLException {
@Override
public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException {
checkClosed();
if (cal == null) {
cal = new GregorianCalendar(TimeZone.getTimeZone("UTC"));//This says whatever date is in UTC
}

LocalDate d = x.toLocalDate();
Calendar c = (Calendar) cal.clone();
Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone();
c.clear();
c.set(d.getYear(), d.getMonthValue() - 1, d.getDayOfMonth(), 0, 0, 0);
parameters[parameterIndex - 1] = encodeObject(c.toInstant());
Expand All @@ -282,12 +282,9 @@ public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLExceptio
@Override
public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException {
checkClosed();
if (cal == null) {
cal = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
}

LocalTime t = x.toLocalTime();
Calendar c = (Calendar) cal.clone();
Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone();
c.clear();
c.set(1970, Calendar.JANUARY, 1, t.getHour(), t.getMinute(), t.getSecond());
parameters[parameterIndex - 1] = encodeObject(c.toInstant());
Expand All @@ -296,12 +293,9 @@ public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLExceptio
@Override
public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException {
checkClosed();
if (cal == null) {
cal = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
}

LocalDateTime ldt = x.toLocalDateTime();
Calendar c = (Calendar) cal.clone();
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());
parameters[parameterIndex - 1] = encodeObject(c.toInstant().atZone(ZoneId.of("UTC")).withNano(x.getNanos()));
Expand Down Expand Up @@ -479,6 +473,8 @@ private static String encodeObject(Object x) throws SQLException {
return "'" + DATETIME_FORMATTER.format(((Timestamp) x).toLocalDateTime()) + "'";
} else if (x instanceof LocalDateTime) {
return "'" + DATETIME_FORMATTER.format((LocalDateTime) x) + "'";
} 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) {
Expand Down
32 changes: 12 additions & 20 deletions jdbc-v2/src/main/java/com/clickhouse/jdbc/ResultSetImpl.java
Original file line number Diff line number Diff line change
@@ -1,24 +1,17 @@
package com.clickhouse.jdbc;

import java.io.ByteArrayInputStream;
import java.io.CharArrayReader;
import java.io.InputStream;
import java.io.Reader;
import java.io.StringReader;
import java.math.BigDecimal;
import java.net.MalformedURLException;
import java.net.URL;
import java.nio.charset.StandardCharsets;
import java.sql.*;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.Calendar;
import java.util.GregorianCalendar;
import java.util.Map;
import java.util.TimeZone;

import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
import com.clickhouse.client.api.metadata.TableSchema;
Expand Down Expand Up @@ -47,7 +40,7 @@ public ResultSetImpl(StatementImpl parentStatement, QueryResponse response, Clic
this.metaData = new com.clickhouse.jdbc.metadata.ResultSetMetaData(this);
this.closed = false;
this.wasNull = false;
this.defaultCalendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
this.defaultCalendar = parentStatement.connection.defaultCalendar;
}

protected ResultSetImpl(ResultSetImpl resultSet) {
Expand All @@ -57,7 +50,7 @@ protected ResultSetImpl(ResultSetImpl resultSet) {
this.metaData = resultSet.metaData;
this.closed = false;
this.wasNull = false;
this.defaultCalendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
this.defaultCalendar = parentStatement.connection.defaultCalendar;
}

private void checkClosed() throws SQLException {
Expand Down Expand Up @@ -1048,16 +1041,16 @@ public Date getDate(int columnIndex, Calendar cal) throws SQLException {
public Date getDate(String columnLabel, Calendar cal) throws SQLException {
checkClosed();
try {
LocalDate d = reader.getLocalDate(columnLabel);
if (d == null) {
ZonedDateTime zdt = reader.getZonedDateTime(columnLabel);
if (zdt == null) {
wasNull = true;
return null;
}
wasNull = false;

Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone();
c.clear();
c.set(d.getYear(), d.getMonthValue() - 1, d.getDayOfMonth(), 0, 0, 0);
c.set(zdt.getYear(), zdt.getMonthValue() - 1, zdt.getDayOfMonth(), 0, 0, 0);
return new Date(c.getTimeInMillis());
} catch (Exception e) {
throw ExceptionUtils.toSqlState(String.format("Method: getDate(\"%s\") encountered an exception.", columnLabel), String.format("SQL: [%s]", parentStatement.getLastSql()), e);
Expand All @@ -1080,7 +1073,7 @@ public Time getTime(String columnLabel, Calendar cal) throws SQLException {
}
wasNull = false;

Calendar c = (Calendar)( cal != null ? cal : defaultCalendar).clone();
Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone();
c.clear();
c.set(1970, Calendar.JANUARY, 1, zdt.getHour(), zdt.getMinute(), zdt.getSecond());
return new Time(c.getTimeInMillis());
Expand All @@ -1098,23 +1091,22 @@ public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException
public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException {
checkClosed();
try {
LocalDateTime localDateTime = reader.getLocalDateTime(columnLabel);
if (localDateTime == null) {
ZonedDateTime zdt = reader.getZonedDateTime(columnLabel);
if (zdt == null) {
wasNull = true;
return null;
}
wasNull = false;

Calendar c = (Calendar) (cal != null ? cal : defaultCalendar).clone();
c.set(localDateTime.getYear(), localDateTime.getMonthValue() - 1, localDateTime.getDayOfMonth(), localDateTime.getHour(), localDateTime.getMinute(),
localDateTime.getSecond());
c.set(zdt.getYear(), zdt.getMonthValue() - 1, zdt.getDayOfMonth(), zdt.getHour(), zdt.getMinute(),
zdt.getSecond());
Timestamp timestamp = new Timestamp(c.getTimeInMillis());
timestamp.setNanos(localDateTime.getNano());
timestamp.setNanos(zdt.getNano());
return timestamp;
} catch (Exception e) {
throw ExceptionUtils.toSqlState(String.format("Method: getTimestamp(\"%s\") encountered an exception.", columnLabel), String.format("SQL: [%s]", parentStatement.getLastSql()), e);
}

}

@Override
Expand Down
Loading

0 comments on commit 4807241

Please sign in to comment.