Skip to content

Commit 584e4ed

Browse files
authored
Merge pull request #1800 from ClickHouse/clientv2_small_fixes_0.6.5
[client-v2] Fix reading arrays
2 parents 18d0ca1 + 549276c commit 584e4ed

File tree

4 files changed

+155
-8
lines changed

4 files changed

+155
-8
lines changed

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

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,8 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm
5353

5454
private TableSchema schema;
5555

56+
private ClickHouseColumn[] columns;
57+
5658
private volatile boolean hasNext = true;
5759

5860
protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema) {
@@ -85,7 +87,7 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
8587
*/
8688
public boolean readRecord(Map<String, Object> record) throws IOException {
8789
boolean firstColumn = true;
88-
for (ClickHouseColumn column : getSchema().getColumns()) {
90+
for (ClickHouseColumn column : columns) {
8991
try {
9092
Object val = binaryStreamReader.readValue(column);
9193
if (val != null) {
@@ -170,6 +172,9 @@ protected void endReached() {
170172

171173
protected void setSchema(TableSchema schema) {
172174
this.schema = schema;
175+
if (schema != null) {
176+
columns = schema.getColumns().toArray(new ClickHouseColumn[0]);
177+
}
173178
}
174179

175180
@Override
@@ -371,8 +376,8 @@ public ClickHouseGeoMultiPolygonValue getGeoMultiPolygon(String colName) {
371376

372377
@Override
373378
public <T> List<T> getList(String colName) {
374-
ClickHouseArrayValue<?> array = readValue(colName);
375-
return null;
379+
BinaryStreamReader.ArrayValue array = readValue(colName);
380+
return array.asList();
376381
}
377382

378383

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

Lines changed: 39 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,10 @@
2020
import java.time.LocalDate;
2121
import java.time.LocalDateTime;
2222
import java.time.ZonedDateTime;
23+
import java.util.ArrayList;
2324
import java.util.Collections;
2425
import java.util.LinkedHashMap;
26+
import java.util.List;
2527
import java.util.Map;
2628
import java.util.TimeZone;
2729
import java.util.UUID;
@@ -146,8 +148,10 @@ private <T> T readValueImpl(ClickHouseColumn column) throws IOException {
146148
case IntervalNanosecond:
147149
return (T) readBigIntegerLE(input, 8, true);
148150
case IPv4:
149-
return (T) Inet4Address.getByAddress(readNBytes(input, 4));
151+
// https://clickhouse.com/docs/en/sql-reference/data-types/ipv4
152+
return (T) Inet4Address.getByAddress(readNBytesLE(input, 4));
150153
case IPv6:
154+
// https://clickhouse.com/docs/en/sql-reference/data-types/ipv6
151155
return (T) Inet6Address.getByAddress(readNBytes(input, 16));
152156
case UUID:
153157
return (T) new UUID(readLongLE(input), readLongLE(input));
@@ -290,6 +294,22 @@ public static byte[] readNBytes(InputStream inputStream, int len) throws IOExcep
290294
return bytes;
291295
}
292296

297+
public static byte[] readNBytesLE(InputStream input, int len) throws IOException {
298+
byte[] bytes = readNBytes(input, len);
299+
300+
int s = 0;
301+
int i = len - 1;
302+
while (s < i) {
303+
byte b = bytes[s];
304+
bytes[s] = bytes[i];
305+
bytes[i] = b;
306+
s++;
307+
i--;
308+
}
309+
310+
return bytes;
311+
}
312+
293313
private ArrayValue readArray(ClickHouseColumn column) throws IOException {
294314
Class<?> itemType = column.getArrayBaseColumn().getDataType().getWiderPrimitiveClass();
295315
int len = readVarInt(input);
@@ -345,6 +365,24 @@ public void set(int index, Object value) {
345365
" value " + value + " of class " + value.getClass().getName(), e);
346366
}
347367
}
368+
369+
private List<?> list = null;
370+
371+
public synchronized <T> List<T> asList() {
372+
if (list == null) {
373+
ArrayList<T> list = new ArrayList<>(length);
374+
for (int i = 0; i < length; i++) {
375+
Object item = get(i);
376+
if (item instanceof ArrayValue) {
377+
list.add((T) ((ArrayValue) item).asList());
378+
} else {
379+
list.add((T) item);
380+
}
381+
}
382+
this.list = list;
383+
}
384+
return (List<T>) list;
385+
}
348386
}
349387

350388
private Map<?,?> readMap(ClickHouseColumn column) throws IOException {

client-v2/src/main/java/com/clickhouse/client/api/metadata/TableSchema.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@ public class TableSchema {
1616

1717
private List<ClickHouseColumn> columns;
1818

19+
private List<ClickHouseColumn> columnsView;
20+
1921
private Map<String, Map<String, Object>> metadata;
2022

2123
private Map<String, Integer> colIndex;
@@ -25,6 +27,7 @@ public class TableSchema {
2527
public TableSchema() {
2628
this.metadata = new HashMap<>();
2729
this.columns = new ArrayList<>();
30+
this.columnsView = Collections.unmodifiableList(this.columns);
2831
this.colIndex = new HashMap<>();
2932
}
3033

@@ -34,7 +37,7 @@ public TableSchema() {
3437
* @return - collection of columns in the table
3538
*/
3639
public List<ClickHouseColumn> getColumns() {
37-
return Collections.unmodifiableList(columns);
40+
return columnsView;
3841
}
3942

4043
public String getDatabaseName() {

client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java

Lines changed: 104 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,10 @@
5050
import java.io.OutputStreamWriter;
5151
import java.math.BigDecimal;
5252
import java.math.BigInteger;
53+
import java.net.Inet4Address;
54+
import java.net.Inet6Address;
55+
import java.net.InetAddress;
56+
import java.net.UnknownHostException;
5357
import java.time.LocalDate;
5458
import java.time.LocalDateTime;
5559
import java.time.ZoneId;
@@ -181,6 +185,39 @@ public void testBigUnsignedInt() throws Exception {
181185
Assert.assertEquals(firstRecord.getBigInteger("i256"), expected256);
182186
}
183187

188+
@Test(groups = {"integration"})
189+
public void testEndianReadingNumbers() throws Exception {
190+
191+
byte[][] numbers = new byte[][] {
192+
new byte[] {0x00, 0x02, 0x00, 0x01},
193+
new byte[] {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08},
194+
new byte[] {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0A, 0x0B, 0x0C, 0x0D, 0x0E, 0x0F, 0x10},
195+
};
196+
197+
198+
for (byte[] number : numbers) {
199+
String typeName = "UInt32";
200+
if (number.length == 8) {
201+
typeName = "UInt64";
202+
} else if (number.length == 16) {
203+
typeName = "UInt128";
204+
}
205+
BigInteger expected = new BigInteger(number);
206+
String sqlQuery = "SELECT to" + typeName + "('" + expected + "') as value1";
207+
System.out.println(sqlQuery);
208+
Records records = client.queryRecords(sqlQuery).get(3, TimeUnit.SECONDS);
209+
GenericRecord firstRecord = records.iterator().next();
210+
211+
if (number.length == 4) {
212+
System.out.println(firstRecord.getLong("value1"));
213+
Assert.assertEquals(firstRecord.getLong("value1"), expected.longValue());
214+
} else {
215+
System.out.println(firstRecord.getBigInteger("value1"));
216+
Assert.assertEquals(firstRecord.getBigInteger("value1"), expected);
217+
}
218+
}
219+
}
220+
184221
@Test(groups = {"integration"})
185222
public void testReadRecordsWithStreamAPI() throws Exception {
186223
final int tables = 10;
@@ -426,7 +463,8 @@ record = reader.next();
426463

427464
private final static List<Function<String, Object>> ARRAY_VALUE_GENERATORS = Arrays.asList(
428465
c ->
429-
RANDOM.ints(10, 0, 100),
466+
RANDOM.ints(10, 0, 100)
467+
.asLongStream().collect(ArrayList::new, ArrayList::add, ArrayList::addAll),
430468
c -> {
431469
List<List<Integer>> values = new ArrayList<>();
432470
for (int i = 0; i < 10; i++) {
@@ -453,9 +491,13 @@ public void testArrayValues() throws Exception {
453491

454492
Map<String, Object> record = reader.next();
455493
Assert.assertNotNull(record);
494+
Map<String, Object> datasetRecord = data.get(0);
456495
long[] col1Values = reader.getLongArray("col1");
457-
System.out.println("col1: " + Arrays.toString(col1Values));
458-
System.out.println("Record: " + record);
496+
Assert.assertEquals(Arrays.stream(col1Values).collect(ArrayList<Long>::new, ArrayList::add,
497+
ArrayList::addAll), datasetRecord.get("col1"));
498+
Assert.assertEquals(reader.getList("col1"), datasetRecord.get("col1"));
499+
List<List<Long>> col2Values = reader.getList("col2");
500+
Assert.assertEquals(col2Values, data.get(0).get("col2"));
459501
}
460502

461503
private final static List<String> MAP_COLUMNS = Arrays.asList(
@@ -574,6 +616,65 @@ public void testNullValues() throws Exception {
574616
}
575617
}
576618

619+
@Test
620+
public void testIPAddresses() throws Exception {
621+
622+
final List<String> columns = Arrays.asList(
623+
"srcV4 IPv4",
624+
"targetV4 IPv4",
625+
"srcV6 IPv6",
626+
"targetV6 IPv6"
627+
628+
);
629+
630+
Random random = new Random();
631+
byte[] ipv4 = new byte[4];
632+
random.nextBytes(ipv4);
633+
InetAddress ipv4src = Inet4Address.getByAddress(ipv4);
634+
random.nextBytes(ipv4);
635+
InetAddress ipv4target = Inet4Address.getByAddress(ipv4);
636+
byte[] ipv6 = new byte[16];
637+
random.nextBytes(ipv6);
638+
InetAddress ipv6src = Inet6Address.getByAddress(ipv6);
639+
random.nextBytes(ipv6);
640+
InetAddress ipv6target = Inet6Address.getByAddress(ipv6);
641+
642+
643+
final List<Supplier<String>> valueGenerators = Arrays.asList(
644+
() -> sq(ipv4src.getHostAddress()),
645+
() -> sq(ipv4target.getHostAddress()),
646+
() -> sq(ipv6src.getHostAddress()),
647+
() -> sq(ipv6target.getHostAddress())
648+
);
649+
650+
final List<Consumer<ClickHouseBinaryFormatReader>> verifiers = new ArrayList<>();
651+
verifiers.add(r -> {
652+
Assert.assertTrue(r.hasValue("srcV4"), "No value for column srcV4 found");
653+
Assert.assertEquals(r.getInet4Address("srcV4"), ipv4src);
654+
Assert.assertEquals(r.getInet4Address(1), ipv4src);
655+
});
656+
657+
verifiers.add(r -> {
658+
Assert.assertTrue(r.hasValue("targetV4"), "No value for column targetV4 found");
659+
Assert.assertEquals(r.getInet4Address("targetV4"), ipv4target);
660+
Assert.assertEquals(r.getInet4Address(2), ipv4target);
661+
});
662+
663+
verifiers.add(r -> {
664+
Assert.assertTrue(r.hasValue("srcV6"), "No value for column src6 found");
665+
Assert.assertEquals(r.getInet6Address("srcV6"), ipv6src);
666+
Assert.assertEquals(r.getInet6Address(3), ipv6src);
667+
});
668+
669+
verifiers.add(r -> {
670+
Assert.assertTrue(r.hasValue("targetV6"), "No value for column targetV6 found");
671+
Assert.assertEquals(r.getInet6Address("targetV6"), ipv6target);
672+
Assert.assertEquals(r.getInet6Address(4), ipv6target);
673+
});
674+
675+
testDataTypes(columns, valueGenerators, verifiers);
676+
}
677+
577678
@Test
578679
public void testDateTimeDataTypes() {
579680
final List<String> columns = Arrays.asList(

0 commit comments

Comments
 (0)