Skip to content

Commit 07b8b1f

Browse files
committed
Merge branch 'main' into jdbc_prepared_stmt_metadata
2 parents eb0762f + 18ed07e commit 07b8b1f

File tree

24 files changed

+1198
-71
lines changed

24 files changed

+1198
-71
lines changed

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,8 @@ public class RowBinaryFormatWriter {
2929

3030
private final Object[] row;
3131

32+
private final boolean defaultSupport;
33+
3234
public RowBinaryFormatWriter(OutputStream out, TableSchema tableSchema, ClickHouseFormat format) {
3335
if (format != ClickHouseFormat.RowBinary && format != ClickHouseFormat.RowBinaryWithDefaults) {
3436
throw new IllegalArgumentException("Only RowBinary and RowBinaryWithDefaults are supported");
@@ -37,6 +39,7 @@ public RowBinaryFormatWriter(OutputStream out, TableSchema tableSchema, ClickHou
3739
this.out = out;
3840
this.tableSchema = tableSchema;
3941
this.row = new Object[tableSchema.getColumns().size()];
42+
this.defaultSupport = format == ClickHouseFormat.RowBinaryWithDefaults;
4043
}
4144

4245
public void setValue(String column, Object value) {
@@ -48,12 +51,11 @@ public void setValue(int colIndex, Object value) {
4851
}
4952

5053
public void commitRow() throws IOException {
51-
5254
List<ClickHouseColumn> columnList = tableSchema.getColumns();
5355
for (int i = 0; i < row.length; i++) {
5456
ClickHouseColumn column = columnList.get(i);
5557

56-
if (RowBinaryFormatSerializer.writeValuePreamble(out, true, column, row[i])) {
58+
if (RowBinaryFormatSerializer.writeValuePreamble(out, defaultSupport, column, row[i])) {
5759
SerializerUtils.serializeData(out, row[i], column);
5860
}
5961
}

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

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -205,9 +205,12 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
205205
return (T) readGeoPolygon();
206206
case MultiPolygon:
207207
return (T) readGeoMultiPolygon();
208+
case MultiLineString:
209+
return (T) readGeoPolygon();
208210
case Ring:
209211
return (T) readGeoRing();
210-
212+
case LineString:
213+
return (T) readGeoRing();
211214
case JSON: // experimental https://clickhouse.com/docs/en/sql-reference/data-types/newjson
212215
if (jsonAsString) {
213216
return (T) readString(input);

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,10 +80,12 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo
8080
serializeTupleData(stream, value, GEO_POINT_TUPLE);
8181
break;
8282
case Ring:
83+
case LineString:
8384
value = value instanceof ClickHouseGeoRingValue ? ((ClickHouseGeoRingValue)value).getValue() : value;
8485
serializeArrayData(stream, value, GEO_RING_ARRAY);
8586
break;
8687
case Polygon:
88+
case MultiLineString:
8789
value = value instanceof ClickHouseGeoPolygonValue ? ((ClickHouseGeoPolygonValue)value).getValue() : value;
8890
serializeArrayData(stream, value, GEO_POLYGON_ARRAY);
8991
break;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ private int refill() throws IOException {
129129

130130
if (buffer.capacity() < uncompressedSize) {
131131
buffer = ByteBuffer.allocate(uncompressedSize);
132-
LOG.warn("Buffer size is too small, reallocate buffer with size: " + uncompressedSize);
132+
LOG.debug("Buffer size is too small, reallocate buffer with size: {}", uncompressedSize);
133133
}
134134
decompressor.decompress(ByteBuffer.wrap(block), offset, buffer, 0, uncompressedSize);
135135
buffer.position(0);

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -380,7 +380,7 @@ public Exception readError(ClassicHttpResponse httpResponse) {
380380

381381
public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Object> requestConfig, LZ4Factory lz4Factory,
382382
IOCallback<OutputStream> writeCallback) throws IOException {
383-
if (timeToPoolVent.get() < System.currentTimeMillis()) {
383+
if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) {
384384
timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT);
385385
poolControl.closeExpired();
386386
}

client-v2/src/test/java/com/clickhouse/client/ClientTests.java

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

33
import com.clickhouse.client.api.Client;
4+
import com.clickhouse.client.api.ClientConfigProperties;
45
import com.clickhouse.client.api.ClientException;
56
import com.clickhouse.client.api.enums.Protocol;
67
import com.clickhouse.client.api.query.GenericRecord;
@@ -108,6 +109,13 @@ public void testPing() {
108109
}
109110
}
110111

112+
@Test
113+
public void testPingUnpooled() {
114+
try (Client client = newClient().enableConnectionPool(false).build()) {
115+
Assert.assertTrue(client.ping());
116+
}
117+
}
118+
111119
@Test
112120
public void testPingFailure() {
113121
try (Client client = new Client.Builder()

0 commit comments

Comments
 (0)