Skip to content

Commit c8a61a1

Browse files
committed
Merge branch 'main' into fix_statement_impl
2 parents 8c3c2d7 + 8a797c9 commit c8a61a1

File tree

18 files changed

+1103
-93
lines changed

18 files changed

+1103
-93
lines changed

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -211,6 +211,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
211211
}
212212
break;
213213
case DateTime32:
214+
case Time:
214215
if (size > 0) {
215216
column.template = ClickHouseOffsetDateTimeValue.ofNull(
216217
column.scale,
@@ -219,6 +220,7 @@ private static ClickHouseColumn update(ClickHouseColumn column) {
219220
}
220221
break;
221222
case DateTime64:
223+
case Time64:
222224
if (size > 0) {
223225
column.scale = Integer.parseInt(column.parameters.get(0));
224226
}

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

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@
55
import java.net.Inet4Address;
66
import java.net.Inet6Address;
77
import java.time.Duration;
8+
import java.time.Instant;
89
import java.time.LocalDate;
910
import java.time.LocalDateTime;
1011
import java.time.LocalTime;
@@ -127,6 +128,8 @@ public enum ClickHouseDataType {
127128
AggregateFunction(String.class, true, true, false, 0, 0, 0, 0, 0, true),
128129
Variant(List.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2A),
129130
Dynamic(Object.class, true, true, false, 0, 0, 0, 0, 0, true, 0x2B),
131+
Time(LocalDateTime.class, true, false, false, 4, 9, 0, 0, 9, false, 0x32), // 0x33 for Time(Timezone)
132+
Time64(LocalDateTime.class, true, false, false, 8, 9, 0, 0, 0, false, 0x34), // 0x35 for Time64(P, Timezone)
130133
;
131134

132135
public static final List<ClickHouseDataType> ORDERED_BY_RANGE_INT_TYPES =
@@ -237,6 +240,9 @@ static Map<ClickHouseDataType, Set<Class<?>>> dataTypeClassMap() {
237240
map.put(IntervalMicrosecond, timeIntervalClasses);
238241
map.put(IntervalNanosecond, timeIntervalClasses);
239242

243+
map.put(Time, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, Instant.class))));
244+
map.put(Time64, Collections.unmodifiableSet(new HashSet<>(Arrays.asList(Integer.class, Long.class, BigInteger.class, Instant.class))));
245+
240246
return map;
241247
}
242248

clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseColumnTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -420,7 +420,8 @@ public boolean isWidenUnsignedTypes() {
420420
// skip advanced types
421421
if (type.isNested() || type == ClickHouseDataType.AggregateFunction
422422
|| type == ClickHouseDataType.SimpleAggregateFunction || type == ClickHouseDataType.Enum
423-
|| type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16) {
423+
|| type == ClickHouseDataType.Nullable || type == ClickHouseDataType.BFloat16 ||
424+
type == ClickHouseDataType.Time || type == ClickHouseDataType.Time64) {
424425
continue;
425426
}
426427

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,6 @@
77
import com.clickhouse.client.api.data_formats.RowBinaryFormatReader;
88
import com.clickhouse.client.api.data_formats.RowBinaryWithNamesAndTypesFormatReader;
99
import com.clickhouse.client.api.data_formats.RowBinaryWithNamesFormatReader;
10-
import com.clickhouse.client.api.data_formats.internal.AbstractBinaryFormatReader;
1110
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
1211
import com.clickhouse.client.api.data_formats.internal.MapBackedRecord;
1312
import com.clickhouse.client.api.data_formats.internal.ProcessParser;
@@ -37,7 +36,6 @@
3736
import com.clickhouse.client.api.transport.Endpoint;
3837
import com.clickhouse.client.api.transport.HttpEndpoint;
3938
import com.clickhouse.client.config.ClickHouseClientOption;
40-
import com.clickhouse.config.ClickHouseOption;
4139
import com.clickhouse.data.ClickHouseColumn;
4240
import com.clickhouse.data.ClickHouseDataType;
4341
import com.clickhouse.data.ClickHouseFormat;
@@ -1575,7 +1573,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15751573
Supplier<QueryResponse> responseSupplier;
15761574

15771575
if (queryParams != null) {
1578-
settings.setOption("statement_params", queryParams);
1576+
settings.setOption(HttpAPIClientHelper.KEY_STATEMENT_PARAMS, queryParams);
15791577
}
15801578
final QuerySettings finalSettings = new QuerySettings(buildRequestSettings(settings.getAllSettings()));
15811579
responseSupplier = () -> {
@@ -2027,6 +2025,7 @@ protected int getOperationTimeout() {
20272025
* @return - set of endpoints
20282026
* @deprecated
20292027
*/
2028+
@Deprecated
20302029
public Set<String> getEndpoints() {
20312030
return endpoints.stream().map(Endpoint::getBaseURL).collect(Collectors.toSet());
20322031
}
@@ -2100,4 +2099,5 @@ private Map<String, Object> buildRequestSettings(Map<String, Object> opSettings)
21002099
requestSettings.putAll(opSettings);
21012100
return requestSettings;
21022101
}
2102+
21032103
}

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

Lines changed: 117 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,19 @@
11
package com.clickhouse.client.api;
22

3+
import java.time.Instant;
4+
import java.time.ZoneId;
5+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
6+
7+
import java.time.Instant;
8+
import java.time.ZoneId;
39
import java.time.format.DateTimeFormatter;
10+
import java.time.format.DateTimeFormatterBuilder;
11+
import java.time.temporal.ChronoField;
12+
import java.util.Objects;
13+
14+
import com.clickhouse.data.ClickHouseDataType;
15+
16+
import static com.clickhouse.client.api.data_formats.internal.BinaryStreamReader.BASES;
417

518
public class DataTypeUtils {
619

@@ -19,4 +32,108 @@ public class DataTypeUtils {
1932
*/
2033
public static DateTimeFormatter DATETIME_WITH_NANOS_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.nnnnnnnnn");
2134

35+
private static final DateTimeFormatter INSTANT_FORMATTER = new DateTimeFormatterBuilder()
36+
.appendValue(ChronoField.INSTANT_SECONDS)
37+
.appendFraction(ChronoField.NANO_OF_SECOND, 9, 9, true)
38+
.toFormatter();
39+
40+
/**
41+
* Formats an {@link Instant} object for use in SQL statements or as query
42+
* parameter.
43+
*
44+
* @param instant
45+
* the Java object to format
46+
* @return a suitable String representation of {@code instant}
47+
* @throws NullPointerException
48+
* if {@code instant} is null
49+
*/
50+
public static String formatInstant(Instant instant) {
51+
return formatInstant(instant, null);
52+
}
53+
54+
/**
55+
* Formats an {@link Instant} object for use in SQL statements or as query
56+
* parameter.
57+
*
58+
* This method uses the {@code dataTypeHint} parameter to find the best
59+
* suitable format for the instant.
60+
*
61+
* @param instant
62+
* the Java object to format
63+
* @param dataTypeHint
64+
* the ClickHouse data type {@code instant} should be used for
65+
* @return a suitable String representation of {@code instant}
66+
* @throws NullPointerException
67+
* if {@code instant} is null
68+
*/
69+
public static String formatInstant(Instant instant, ClickHouseDataType dataTypeHint) {
70+
return formatInstant(instant, dataTypeHint, null);
71+
}
72+
73+
/**
74+
* Formats an {@link Instant} object for use in SQL statements or as query
75+
* parameter.
76+
*
77+
* This method uses the {@code dataTypeHint} parameter to find the best
78+
* suitable format for the instant.
79+
*
80+
* For <em>some</em> formatting operations, providing a {@code timeZone} is
81+
* mandatory, e.g. for {@link ClickHouseDataType#Date}.
82+
*
83+
* @param instant
84+
* the Java object to format
85+
* @param dataTypeHint
86+
* the ClickHouse data type {@code object} should be used for
87+
* @param timeZone
88+
* the time zone to be used when formatting the instant for use
89+
* in non-time-zone-based ClickHouse data types
90+
* @return a suitable String representation of {@code object}, or the empty
91+
* String for {@code null} objects
92+
* @throws NullPointerException
93+
* if {@code instant} is null
94+
*/
95+
public static String formatInstant(Instant instant, ClickHouseDataType dataTypeHint,
96+
ZoneId timeZone)
97+
{
98+
Objects.requireNonNull(instant, "Instant required for formatInstant");
99+
if (dataTypeHint == null) {
100+
return formatInstantDefault(instant);
101+
}
102+
switch (dataTypeHint) {
103+
case Date:
104+
case Date32:
105+
Objects.requireNonNull(
106+
timeZone,
107+
"TimeZone required for formatting Instant for '" + dataTypeHint + "' use");
108+
return DATE_FORMATTER.format(
109+
instant.atZone(timeZone).toLocalDate());
110+
case DateTime:
111+
case DateTime32:
112+
return String.valueOf(instant.getEpochSecond());
113+
default:
114+
return formatInstantDefault(instant);
115+
}
116+
}
117+
118+
private static String formatInstantDefault(Instant instant) {
119+
return INSTANT_FORMATTER.format(instant);
120+
}
121+
122+
public static Instant instantFromTime64Integer(int precision, long value) {
123+
int nanoSeconds = 0;
124+
if (precision > 0) {
125+
int factor = BinaryStreamReader.BASES[precision];
126+
nanoSeconds = (int) (value % factor);
127+
value /= factor;
128+
if (nanoSeconds < 0) {
129+
nanoSeconds += factor;
130+
value--;
131+
}
132+
if (nanoSeconds > 0L) {
133+
nanoSeconds *= BASES[9 - precision];
134+
}
135+
}
136+
137+
return Instant.ofEpochSecond(value, nanoSeconds);
138+
}
22139
}

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

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,6 @@
33
import com.clickhouse.client.api.data_formats.internal.SerializerUtils;
44
import com.clickhouse.data.ClickHouseColumn;
55
import com.clickhouse.data.ClickHouseDataType;
6-
import com.clickhouse.data.ClickHouseFormat;
76
import com.clickhouse.data.format.BinaryStreamUtils;
87

98
import java.io.IOException;
@@ -132,7 +131,7 @@ public void writeFixedString(String value, int len) throws IOException {
132131
}
133132

134133
public void writeDate(ZonedDateTime value) throws IOException {
135-
SerializerUtils.writeDate(out, value, ZoneId.of("UTC"));
134+
SerializerUtils.writeDate(out, value, value.getZone());
136135
}
137136

138137
public void writeDate32(ZonedDateTime value, ZoneId targetTz) throws IOException {

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

Lines changed: 16 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22

33
import com.clickhouse.client.api.ClientConfigProperties;
44
import com.clickhouse.client.api.ClientException;
5+
import com.clickhouse.client.api.DataTypeUtils;
56
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
67
import com.clickhouse.client.api.internal.MapUtils;
78
import com.clickhouse.client.api.internal.ServerSettings;
@@ -304,6 +305,8 @@ protected void setSchema(TableSchema schema) {
304305
case Enum16:
305306
case Variant:
306307
case Dynamic:
308+
case Time:
309+
case Time64:
307310
this.convertions[i] = NumberConverter.NUMBER_CONVERTERS;
308311
break;
309312
default:
@@ -452,6 +455,10 @@ public Instant getInstant(String colName) {
452455
ZonedDateTime dateTime = (ZonedDateTime) colValue;
453456
return dateTime.toInstant();
454457
}
458+
case Time:
459+
return Instant.ofEpochSecond(getLong(colName));
460+
case Time64:
461+
return DataTypeUtils.instantFromTime64Integer(column.getScale(), getLong(colName));
455462
default:
456463
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
457464
}
@@ -472,7 +479,7 @@ public ZonedDateTime getZonedDateTime(String colName) {
472479
case Date32:
473480
return readValue(colName);
474481
default:
475-
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to Instant");
482+
throw new ClientException("Column of type " + column.getDataType() + " cannot be converted to ZonedDateTime");
476483
}
477484
}
478485

@@ -671,7 +678,7 @@ public BigDecimal getBigDecimal(int index) {
671678

672679
@Override
673680
public Instant getInstant(int index) {
674-
return readValue(index);
681+
return getInstant(schema.columnIndexToName(index));
675682
}
676683

677684
@Override
@@ -706,22 +713,22 @@ public UUID getUUID(int index) {
706713

707714
@Override
708715
public ClickHouseGeoPointValue getGeoPoint(int index) {
709-
return readValue(index);
716+
return getGeoPoint(schema.columnIndexToName(index));
710717
}
711718

712719
@Override
713720
public ClickHouseGeoRingValue getGeoRing(int index) {
714-
return readValue(index);
721+
return getGeoRing(schema.columnIndexToName(index));
715722
}
716723

717724
@Override
718725
public ClickHouseGeoPolygonValue getGeoPolygon(int index) {
719-
return readValue(index);
726+
return getGeoPolygon(schema.columnIndexToName(index));
720727
}
721728

722729
@Override
723730
public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(int index) {
724-
return readValue(index);
731+
return getGeoMultiPolygon(schema.columnIndexToName(index));
725732
}
726733

727734
@Override
@@ -803,11 +810,7 @@ public LocalDate getLocalDate(String colName) {
803810

804811
@Override
805812
public LocalDate getLocalDate(int index) {
806-
Object value = readValue(index);
807-
if (value instanceof ZonedDateTime) {
808-
return ((ZonedDateTime) value).toLocalDate();
809-
}
810-
return (LocalDate) value;
813+
return getLocalDate(schema.columnIndexToName(index));
811814
}
812815

813816
@Override
@@ -821,11 +824,7 @@ public LocalDateTime getLocalDateTime(String colName) {
821824

822825
@Override
823826
public LocalDateTime getLocalDateTime(int index) {
824-
Object value = readValue(index);
825-
if (value instanceof ZonedDateTime) {
826-
return ((ZonedDateTime) value).toLocalDateTime();
827-
}
828-
return (LocalDateTime) value;
827+
return getLocalDateTime(schema.columnIndexToName(index));
829828
}
830829

831830
@Override
@@ -839,11 +838,7 @@ public OffsetDateTime getOffsetDateTime(String colName) {
839838

840839
@Override
841840
public OffsetDateTime getOffsetDateTime(int index) {
842-
Object value = readValue(index);
843-
if (value instanceof ZonedDateTime) {
844-
return ((ZonedDateTime) value).toOffsetDateTime();
845-
}
846-
return (OffsetDateTime) value;
841+
return getOffsetDateTime(schema.columnIndexToName(index));
847842
}
848843

849844
@Override

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

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
104104
if (column.isNullable()) {
105105
int isNull = readByteOrEOF(input);
106106
if (isNull == 1) { // is Null?
107-
return (T) null;
107+
return null;
108108
}
109109
}
110110

@@ -186,6 +186,10 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
186186
return convertDateTime(readDateTime32(timezone), typeHint);
187187
case DateTime64:
188188
return convertDateTime(readDateTime64(scale, timezone), typeHint);
189+
case Time:
190+
return (T) (Integer) readIntLE();
191+
case Time64:
192+
return (T) (Long) (readLongLE());
189193
case IntervalYear:
190194
case IntervalQuarter:
191195
case IntervalMonth:
@@ -588,7 +592,7 @@ public static byte[] readNBytesLE(InputStream input, byte[] buffer, int offset,
588592

589593
return bytes;
590594
}
591-
595+
592596
/**
593597
* Reads a array into an ArrayValue object.
594598
* @param column - column information
@@ -964,7 +968,7 @@ private ZonedDateTime readDateTime32(TimeZone tz) throws IOException {
964968
*/
965969
public static ZonedDateTime readDateTime32(InputStream input, byte[] buff, TimeZone tz) throws IOException {
966970
long time = readUnsignedIntLE(input, buff);
967-
return LocalDateTime.ofInstant(Instant.ofEpochSecond(Math.max(time, 0L)), tz.toZoneId()).atZone(tz.toZoneId());
971+
return Instant.ofEpochSecond(Math.max(time, 0L)).atZone(tz.toZoneId());
968972
}
969973

970974
/**

0 commit comments

Comments
 (0)