Skip to content

Commit 629c0be

Browse files
committed
Merge branch 'main' into v2_support_dynamic_and_json
2 parents c6335d7 + b9384fb commit 629c0be

File tree

13 files changed

+232
-47
lines changed

13 files changed

+232
-47
lines changed

clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseUtils.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -362,7 +362,7 @@ public static List<Path> findFiles(String pattern, String... paths) throws IOExc
362362
pattern.chars().anyMatch(
363363
value -> {
364364
if (value < ' ' || reservedCharsWindows.indexOf(value) != -1) {
365-
throw new IllegalArgumentException("File path contains reserved character <%s>".formatted((char) value));
365+
throw new IllegalArgumentException(String.format("File path contains reserved character <%s>", value));
366366
}
367367
return false;
368368
}

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1882,8 +1882,9 @@ public TableSchema getTableSchemaFromQuery(String sql) {
18821882
private TableSchema getTableSchemaImpl(String describeQuery, String name, String originalQuery, String database) {
18831883
int operationTimeout = getOperationTimeout();
18841884

1885-
try (QueryResponse response = operationTimeout == 0 ? query(describeQuery).get() :
1886-
query(describeQuery).get(getOperationTimeout(), TimeUnit.SECONDS)) {
1885+
QuerySettings settings = new QuerySettings().setDatabase(database);
1886+
try (QueryResponse response = operationTimeout == 0 ? query(describeQuery, settings).get() :
1887+
query(describeQuery, settings).get(getOperationTimeout(), TimeUnit.SECONDS)) {
18871888
return TableSchemaParser.readTSKV(response.getInputStream(), name, originalQuery, database);
18881889
} catch (TimeoutException e) {
18891890
throw new ClientException("Operation has likely timed out after " + getOperationTimeout() + " seconds.", e);
@@ -2006,6 +2007,7 @@ private <T> CompletableFuture<T> runAsyncOperation(Supplier<T> resultSupplier, M
20062007
return isAsync ? CompletableFuture.supplyAsync(resultSupplier, sharedOperationExecutor) : CompletableFuture.completedFuture(resultSupplier.get());
20072008
}
20082009

2010+
@Override
20092011
public String toString() {
20102012
return "Client{" +
20112013
"endpoints=" + endpoints +

client-v2/src/main/java/com/clickhouse/client/api/data_formats/ClickHouseBinaryFormatReader.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,7 @@
1111
import java.math.BigInteger;
1212
import java.net.Inet4Address;
1313
import java.net.Inet6Address;
14-
import java.time.Duration;
15-
import java.time.Instant;
16-
import java.time.LocalDate;
17-
import java.time.LocalDateTime;
18-
import java.time.ZonedDateTime;
14+
import java.time.*;
1915
import java.util.List;
2016
import java.util.Map;
2117
import java.util.UUID;
@@ -535,6 +531,10 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable {
535531

536532
LocalDateTime getLocalDateTime(int index);
537533

534+
OffsetDateTime getOffsetDateTime(String colName);
535+
536+
OffsetDateTime getOffsetDateTime(int index);
537+
538538
TableSchema getSchema();
539539

540540
ClickHouseBitmap getClickHouseBitmap(String colName);

client-v2/src/main/java/com/clickhouse/client/api/data_formats/RowBinaryFormatSerializer.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,8 @@
1212
import java.math.BigInteger;
1313
import java.net.Inet4Address;
1414
import java.net.Inet6Address;
15+
import java.time.Instant;
16+
import java.time.OffsetDateTime;
1517
import java.time.ZoneId;
1618
import java.time.ZonedDateTime;
1719

@@ -145,6 +147,22 @@ public void writeDateTime64(ZonedDateTime value, int scale, ZoneId targetTz) thr
145147
SerializerUtils.writeDateTime64(out, value, scale, targetTz);
146148
}
147149

150+
public void writeDateTime32(OffsetDateTime value) throws IOException {
151+
SerializerUtils.writeDateTime32(out, value, null);
152+
}
153+
154+
public void writeDateTime64(OffsetDateTime value, int scale) throws IOException {
155+
SerializerUtils.writeDateTime64(out, value, scale, null);
156+
}
157+
158+
public void writeDateTime32(Instant value) throws IOException {
159+
SerializerUtils.writeDateTime32(out, value, null);
160+
}
161+
162+
public void writeDateTime64(Instant value, int scale) throws IOException {
163+
SerializerUtils.writeDateTime64(out, value, scale, null);
164+
}
165+
148166
public void writeEnum8(byte value) throws IOException {
149167
BinaryStreamUtils.writeEnum8(out, value);
150168
}

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java

Lines changed: 33 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -27,12 +27,7 @@
2727
import java.math.BigInteger;
2828
import java.net.Inet4Address;
2929
import java.net.Inet6Address;
30-
import java.time.Duration;
31-
import java.time.Instant;
32-
import java.time.LocalDate;
33-
import java.time.LocalDateTime;
34-
import java.time.ZoneOffset;
35-
import java.time.ZonedDateTime;
30+
import java.time.*;
3631
import java.time.format.DateTimeFormatter;
3732
import java.time.temporal.ChronoUnit;
3833
import java.util.*;
@@ -221,7 +216,7 @@ protected void endReached() {
221216

222217
protected void setSchema(TableSchema schema) {
223218
this.schema = schema;
224-
this.columns = schema.getColumns().toArray(new ClickHouseColumn[0]);
219+
this.columns = schema.getColumns().toArray(ClickHouseColumn.EMPTY_ARRAY);
225220
this.convertions = new Map[columns.length];
226221

227222
for (int i = 0; i < columns.length; i++) {
@@ -288,7 +283,7 @@ public static String readAsString(Object value, ClickHouseColumn column) {
288283
ClickHouseDataType dataType = column.getDataType();
289284
ZonedDateTime zdt = (ZonedDateTime) value;
290285
if (dataType == ClickHouseDataType.Date) {
291-
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER).toString();
286+
return zdt.format(com.clickhouse.client.api.DataTypeUtils.DATE_FORMATTER);
292287
}
293288
return value.toString();
294289
} else if (value instanceof BinaryStreamReader.EnumValue) {
@@ -389,11 +384,17 @@ public Instant getInstant(String colName) {
389384
return data.atStartOfDay().toInstant(ZoneOffset.UTC);
390385
case DateTime:
391386
case DateTime64:
392-
LocalDateTime dateTime = readValue(colName);
393-
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
394-
387+
Object colValue = readValue(colName);
388+
if (colValue instanceof LocalDateTime) {
389+
LocalDateTime dateTime = (LocalDateTime) colValue;
390+
return dateTime.toInstant(column.getTimeZone().toZoneId().getRules().getOffset(dateTime));
391+
} else {
392+
ZonedDateTime dateTime = (ZonedDateTime) colValue;
393+
return dateTime.toInstant();
394+
}
395+
default:
396+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
395397
}
396-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
397398
}
398399

399400
@Override
@@ -406,9 +407,9 @@ public ZonedDateTime getZonedDateTime(String colName) {
406407
case Date:
407408
case Date32:
408409
return readValue(colName);
410+
default:
411+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
409412
}
410-
411-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
412413
}
413414

414415
@Override
@@ -747,6 +748,24 @@ public LocalDateTime getLocalDateTime(int index) {
747748
return (LocalDateTime) value;
748749
}
749750

751+
@Override
752+
public OffsetDateTime getOffsetDateTime(String colName) {
753+
Object value = readValue(colName);
754+
if (value instanceof ZonedDateTime) {
755+
return ((ZonedDateTime) value).toOffsetDateTime();
756+
}
757+
return (OffsetDateTime) value;
758+
}
759+
760+
@Override
761+
public OffsetDateTime getOffsetDateTime(int index) {
762+
Object value = readValue(index);
763+
if (value instanceof ZonedDateTime) {
764+
return ((ZonedDateTime) value).toOffsetDateTime();
765+
}
766+
return (OffsetDateTime) value;
767+
}
768+
750769
@Override
751770
public ClickHouseBitmap getClickHouseBitmap(String colName) {
752771
return readValue(colName);

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/BinaryReaderBackedRecord.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -354,6 +354,16 @@ public LocalDateTime getLocalDateTime(int index) {
354354
return reader.getLocalDateTime(index);
355355
}
356356

357+
@Override
358+
public OffsetDateTime getOffsetDateTime(String colName) {
359+
return reader.getOffsetDateTime(colName);
360+
}
361+
362+
@Override
363+
public OffsetDateTime getOffsetDateTime(int index) {
364+
return reader.getOffsetDateTime(index);
365+
}
366+
357367
@Override
358368
public Object getObject(String colName) {
359369
return reader.readValue(colName);

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/MapBackedRecord.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -476,6 +476,24 @@ public LocalDateTime getLocalDateTime(int index) {
476476
return (LocalDateTime) value;
477477
}
478478

479+
@Override
480+
public OffsetDateTime getOffsetDateTime(String colName) {
481+
Object value = readValue(colName);
482+
if (value instanceof ZonedDateTime) {
483+
return ((ZonedDateTime) value).toOffsetDateTime();
484+
}
485+
return (OffsetDateTime) value;
486+
}
487+
488+
@Override
489+
public OffsetDateTime getOffsetDateTime(int index) {
490+
Object value = readValue(index);
491+
if (value instanceof ZonedDateTime) {
492+
return ((ZonedDateTime) value).toOffsetDateTime();
493+
}
494+
return (OffsetDateTime) value;
495+
}
496+
479497
@Override
480498
public ClickHouseBitmap getClickHouseBitmap(String colName) {
481499
return readValue(colName);

client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/SerializerUtils.java

Lines changed: 26 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,7 @@
2929
import java.net.Inet4Address;
3030
import java.net.Inet6Address;
3131
import java.sql.Timestamp;
32-
import java.time.LocalDate;
33-
import java.time.LocalDateTime;
34-
import java.time.ZoneId;
35-
import java.time.ZonedDateTime;
32+
import java.time.*;
3633
import java.util.Arrays;
3734
import java.util.Collections;
3835
import java.util.HashMap;
@@ -1006,16 +1003,22 @@ public static void writeDateTime32(OutputStream output, Object value, ZoneId tar
10061003
}
10071004

10081005
public static void writeDateTime(OutputStream output, Object value, ZoneId targetTz) throws IOException {
1009-
long ts = 0;
1006+
long ts;
10101007
if (value instanceof LocalDateTime) {
10111008
LocalDateTime dt = (LocalDateTime) value;
10121009
ts = dt.atZone(targetTz).toEpochSecond();
10131010
} else if (value instanceof ZonedDateTime) {
10141011
ZonedDateTime dt = (ZonedDateTime) value;
1015-
ts = dt.withZoneSameInstant(targetTz).toEpochSecond();
1012+
ts = dt.toEpochSecond();
10161013
} else if (value instanceof Timestamp) {
10171014
Timestamp t = (Timestamp) value;
10181015
ts = t.toLocalDateTime().atZone(targetTz).toEpochSecond();
1016+
} else if(value instanceof OffsetDateTime) {
1017+
OffsetDateTime dt = (OffsetDateTime) value;
1018+
ts = dt.toEpochSecond();
1019+
} else if (value instanceof Instant) {
1020+
Instant dt = (Instant) value;
1021+
ts = dt.getEpochSecond();
10191022
} else {
10201023
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
10211024
}
@@ -1028,20 +1031,30 @@ public static void writeDateTime64(OutputStream output, Object value, int scale,
10281031
throw new IllegalArgumentException("Invalid scale value '" + scale + "'");
10291032
}
10301033

1031-
long ts = 0;
1032-
long nano = 0;
1034+
long ts;
1035+
long nano;
10331036
if (value instanceof LocalDateTime) {
1034-
ZonedDateTime dt = ((LocalDateTime) value).atZone(targetTz);
1035-
ts = dt.toEpochSecond();
1036-
nano = dt.getNano();
1037+
LocalDateTime dt = (LocalDateTime) value;
1038+
ZonedDateTime zdt = dt.atZone(targetTz);
1039+
ts = zdt.toEpochSecond();
1040+
nano = zdt.getNano();
10371041
} else if (value instanceof ZonedDateTime) {
1038-
ZonedDateTime dt = ((ZonedDateTime) value).withZoneSameInstant(targetTz);
1042+
ZonedDateTime dt = (ZonedDateTime) value;
10391043
ts = dt.toEpochSecond();
10401044
nano = dt.getNano();
10411045
} else if (value instanceof Timestamp) {
1042-
ZonedDateTime dt = ((Timestamp) value).toLocalDateTime().atZone(targetTz);
1046+
Timestamp dt = (Timestamp) value;
1047+
ZonedDateTime zdt = dt.toLocalDateTime().atZone(targetTz);
1048+
ts = zdt.toEpochSecond();
1049+
nano = zdt.getNano();
1050+
} else if (value instanceof OffsetDateTime) {
1051+
OffsetDateTime dt = (OffsetDateTime) value;
10431052
ts = dt.toEpochSecond();
10441053
nano = dt.getNano();
1054+
} else if (value instanceof Instant) {
1055+
Instant dt = (Instant) value;
1056+
ts = dt.getEpochSecond();
1057+
nano = dt.getNano();
10451058
} else {
10461059
throw new IllegalArgumentException("Cannot convert " + value + " to DataTime");
10471060
}

client-v2/src/main/java/com/clickhouse/client/api/query/GenericRecord.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -487,6 +487,10 @@ public interface GenericRecord {
487487

488488
LocalDateTime getLocalDateTime(int index);
489489

490+
OffsetDateTime getOffsetDateTime(String colName);
491+
492+
OffsetDateTime getOffsetDateTime(int index);
493+
490494
Object getObject(String colName);
491495

492496
Object getObject(int index);

client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@
4848
import java.time.Instant;
4949
import java.time.ZoneId;
5050
import java.time.ZonedDateTime;
51+
import java.time.temporal.ChronoField;
5152
import java.util.ArrayList;
5253
import java.util.Arrays;
5354
import java.util.Collections;
@@ -105,8 +106,7 @@ protected Client.Builder newClient() {
105106
.useHttpCompression(useHttpCompression)
106107
.setDefaultDatabase(ClickHouseServerForTest.getDatabase())
107108
.serverSetting(ServerSettings.ASYNC_INSERT, "0")
108-
.serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1")
109-
.useNewImplementation(System.getProperty("client.tests.useNewImplementation", "true").equals("true"));
109+
.serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1");
110110
}
111111

112112
@AfterMethod(groups = { "integration" })
@@ -140,7 +140,6 @@ public void insertSimplePOJOs() throws Exception {
140140
assertEquals(response.getQueryId(), uuid);
141141
}
142142

143-
144143
@Test(groups = { "integration" }, enabled = true)
145144
public void insertPOJOWithJSON() throws Exception {
146145
if (isCloud()) {
@@ -163,7 +162,7 @@ public void insertPOJOWithJSON() throws Exception {
163162
client.execute("DROP TABLE IF EXISTS " + tableName, commandSettings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS);
164163
client.execute(createSQL, commandSettings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS);
165164

166-
client.register(PojoWithJSON.class, client.getTableSchema(tableName, "default"));
165+
client.register(PojoWithJSON.class, client.getTableSchema(tableName));
167166
PojoWithJSON pojo = new PojoWithJSON();
168167
pojo.setEventPayload(originalJsonStr);
169168
List<Object> data = Arrays.asList(pojo);
@@ -190,7 +189,7 @@ public void insertPOJOAndReadBack() throws Exception {
190189

191190
initTable(tableName, createSQL);
192191

193-
client.register(SamplePOJO.class, client.getTableSchema(tableName, "default"));
192+
client.register(SamplePOJO.class, client.getTableSchema(tableName));
194193

195194
System.out.println("Inserting POJO: " + pojo);
196195
try (InsertResponse response = client.insert(tableName, Collections.singletonList(pojo), settings).get(EXECUTE_CMD_TIMEOUT, TimeUnit.SECONDS)) {
@@ -213,6 +212,12 @@ public void insertPOJOAndReadBack() throws Exception {
213212
Assert.assertEquals(reader.getDouble("float64"), pojo.getFloat64());
214213
Assert.assertEquals(reader.getString("string"), pojo.getString());
215214
Assert.assertEquals(reader.getString("fixedString"), pojo.getFixedString());
215+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime").isEqual(pojo.getZonedDateTime().withNano(0)));
216+
Assert.assertTrue(reader.getZonedDateTime("zonedDateTime64").isEqual(pojo.getZonedDateTime64()));
217+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime").isEqual(pojo.getOffsetDateTime().withNano(0)));
218+
Assert.assertTrue(reader.getOffsetDateTime("offsetDateTime64").isEqual(pojo.getOffsetDateTime64()));
219+
Assert.assertEquals(reader.getInstant("instant"), pojo.getInstant().with(ChronoField.MICRO_OF_SECOND, 0));
220+
Assert.assertEquals(reader.getInstant("instant64"), pojo.getInstant64());
216221
}
217222
}
218223

@@ -226,7 +231,7 @@ public void testInsertingPOJOWithNullValueForNonNullableColumn() throws Exceptio
226231

227232
initTable(tableName, createSQL);
228233

229-
client.register(SamplePOJO.class, client.getTableSchema(tableName, "default"));
234+
client.register(SamplePOJO.class, client.getTableSchema(tableName));
230235

231236
try (InsertResponse response = client.insert(tableName, Collections.singletonList(pojo), settings).get(30, TimeUnit.SECONDS)) {
232237
fail("Should have thrown an exception");
@@ -457,12 +462,12 @@ public void testWriter() throws Exception {
457462
if (row[4] == null) {
458463
formatWriter.writeDefault();
459464
} else {
460-
formatWriter.writeString((String) row[4]);
465+
formatWriter.writeDateTime((ZonedDateTime) row[4], null);
461466
}
462467
if (row[5] == null) {
463468
formatWriter.writeDefault();
464469
} else {
465-
formatWriter.writeInt8((byte) row[5]);
470+
formatWriter.writeInt8(((Integer) row[5]).byteValue());
466471
}
467472
}
468473
}, ClickHouseFormat.RowBinaryWithDefaults, new InsertSettings()).get()) {

0 commit comments

Comments
 (0)