Skip to content

Commit 54753c8

Browse files
authored
Merge pull request #1995 from ClickHouse/v2_returning_arrays
[client-v2] Fix reading Array(Uint64)
2 parents 277d1fd + a075340 commit 54753c8

File tree

8 files changed

+163
-23
lines changed

8 files changed

+163
-23
lines changed

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

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -281,6 +281,13 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable {
281281
*/
282282
double[] getDoubleArray(String colName);
283283

284+
/**
285+
*
286+
* @param colName
287+
* @return
288+
*/
289+
boolean[] getBooleanArray(String colName);
290+
284291
/**
285292
* Reads column with name `colName` as a string.
286293
*
@@ -503,6 +510,8 @@ public interface ClickHouseBinaryFormatReader extends AutoCloseable {
503510
*/
504511
double[] getDoubleArray(int index);
505512

513+
boolean[] getBooleanArray(int index);
514+
506515
Object[] getTuple(int index);
507516

508517
Object[] getTuple(String colName);

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

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,7 @@
44
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
55
import com.clickhouse.client.api.query.QuerySettings;
66
import com.clickhouse.data.ClickHouseColumn;
7+
import com.clickhouse.data.ClickHouseDataType;
78

89
import java.io.EOFException;
910
import java.io.IOException;
@@ -58,13 +59,24 @@ private boolean readBlock() throws IOException {
5859
for (int i = 0; i < nColumns; i++) {
5960
ClickHouseColumn column = ClickHouseColumn.of(BinaryStreamReader.readString(input),
6061
BinaryStreamReader.readString(input));
62+
6163
names.add(column.getColumnName());
6264
types.add(column.getDataType().name());
6365

6466
List<Object> values = new ArrayList<>(nRows);
65-
for (int j = 0; j < nRows; j++) {
66-
Object value = binaryStreamReader.readValue(column);
67-
values.add(value);
67+
if (column.isArray()) {
68+
int[] sizes = new int[nRows];
69+
for (int j = 0; j < nRows; j++) {
70+
sizes[j] = Math.toIntExact(binaryStreamReader.readLongLE());
71+
}
72+
for (int j = 0; j < nRows; j++) {
73+
values.add(binaryStreamReader.readArrayItem(column.getNestedColumns().get(0), sizes[0]));
74+
}
75+
} else {
76+
for (int j = 0; j < nRows; j++) {
77+
Object value = binaryStreamReader.readValue(column);
78+
values.add(value);
79+
}
6880
}
6981
currentBlock.add(values);
7082
}

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -511,6 +511,11 @@ public double[] getDoubleArray(String colName) {
511511
return getPrimitiveArray(colName);
512512
}
513513

514+
@Override
515+
public boolean[] getBooleanArray(String colName) {
516+
return getPrimitiveArray(colName);
517+
}
518+
514519
@Override
515520
public boolean hasValue(int colIndex) {
516521
return currentRecord.containsKey(getSchema().indexToName(colIndex - 1));
@@ -646,6 +651,11 @@ public double[] getDoubleArray(int index) {
646651
return getPrimitiveArray(schema.indexToName(index));
647652
}
648653

654+
@Override
655+
public boolean[] getBooleanArray(int index) {
656+
return getPrimitiveArray(schema.indexToName(index));
657+
}
658+
649659
@Override
650660
public Object[] getTuple(int index) {
651661
return readValue(index);

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
@@ -158,6 +158,11 @@ public double[] getDoubleArray(String colName) {
158158
return reader.getDoubleArray(colName);
159159
}
160160

161+
@Override
162+
public boolean[] getBooleanArray(String colName) {
163+
return reader.getBooleanArray(colName);
164+
}
165+
161166
@Override
162167
public String getString(int index) {
163168
return reader.getString(index);
@@ -298,6 +303,11 @@ public double[] getDoubleArray(int index) {
298303
return reader.getDoubleArray(index);
299304
}
300305

306+
@Override
307+
public boolean[] getBooleanArray(int index) {
308+
return reader.getBooleanArray(index);
309+
}
310+
301311
@Override
302312
public Object[] getTuple(int index) {
303313
return reader.getTuple(index);

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

Lines changed: 44 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,6 @@
2828
import java.util.TimeZone;
2929
import java.util.UUID;
3030

31-
import static com.clickhouse.data.ClickHouseDataType.toObjectType;
32-
3331
/**
3432
* This class is not thread safe and should not be shared between multiple threads.
3533
* Internally it may use a shared buffer to read data from the input stream.
@@ -521,24 +519,59 @@ public static byte[] readNBytesLE(InputStream input, byte[] buffer, int offset,
521519
* @throws IOException when IO error occurs
522520
*/
523521
public ArrayValue readArray(ClickHouseColumn column) throws IOException {
524-
Class<?> itemType = column.getArrayBaseColumn().getDataType().getWiderPrimitiveClass();
525-
if (column.getArrayBaseColumn().isNullable()) {
526-
itemType = toObjectType(itemType);
527-
}
528522
int len = readVarInt(input);
529-
ArrayValue array = new ArrayValue(column.getArrayNestedLevel() > 1 ? ArrayValue.class : itemType, len);
530-
531523
if (len == 0) {
532-
return array;
524+
return new ArrayValue(Object.class, 0);
533525
}
534526

535-
for (int i = 0; i < len; i++) {
536-
array.set(i, readValue(column.getNestedColumns().get(0)));
527+
ArrayValue array;
528+
ClickHouseColumn itemTypeColumn = column.getNestedColumns().get(0);
529+
if (column.getArrayNestedLevel() == 1) {
530+
array = readArrayItem(itemTypeColumn, len);
531+
532+
} else {
533+
array = new ArrayValue(ArrayValue.class, len);
534+
for (int i = 0; i < len; i++) {
535+
array.set(i, readArray(itemTypeColumn));
536+
}
537537
}
538538

539539
return array;
540540
}
541541

542+
public ArrayValue readArrayItem(ClickHouseColumn itemTypeColumn, int len) throws IOException {
543+
ArrayValue array;
544+
if (itemTypeColumn.isNullable()) {
545+
array = new ArrayValue(Object.class, len);
546+
for (int i = 0; i < len; i++) {
547+
array.set(i, readValue(itemTypeColumn));
548+
}
549+
} else {
550+
Object firstValue = readValue(itemTypeColumn);
551+
Class<?> itemClass = firstValue.getClass();
552+
if (firstValue instanceof Byte) {
553+
itemClass = byte.class;
554+
} else if (firstValue instanceof Character) {
555+
itemClass = char.class;
556+
} else if (firstValue instanceof Short) {
557+
itemClass = short.class;
558+
} else if (firstValue instanceof Integer) {
559+
itemClass = int.class;
560+
} else if (firstValue instanceof Long) {
561+
itemClass = long.class;
562+
} else if (firstValue instanceof Boolean) {
563+
itemClass = boolean.class;
564+
}
565+
566+
array = new ArrayValue(itemClass, len);
567+
array.set(0, firstValue);
568+
for (int i = 1; i < len; i++) {
569+
array.set(i, readValue(itemTypeColumn));
570+
}
571+
}
572+
return array;
573+
}
574+
542575
public void skipValue(ClickHouseColumn column) throws IOException {
543576
readValue(column, null);
544577
}
@@ -557,8 +590,6 @@ public static class ArrayValue {
557590

558591
try {
559592
if (itemType.isArray()) {
560-
array = Array.newInstance(ArrayValue.class, length);
561-
} else if (itemType == List.class) {
562593
array = Array.newInstance(Object[].class, length);
563594
} else {
564595
array = Array.newInstance(itemType, length);

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -286,6 +286,11 @@ public double[] getDoubleArray(String colName) {
286286
return getPrimitiveArray(colName);
287287
}
288288

289+
@Override
290+
public boolean[] getBooleanArray(String colName) {
291+
return getPrimitiveArray(colName);
292+
}
293+
289294
@Override
290295
public boolean hasValue(int colIndex) {
291296
return record.containsKey(schema.indexToName(colIndex));
@@ -426,6 +431,11 @@ public double[] getDoubleArray(int index) {
426431
return getPrimitiveArray(schema.indexToName(index));
427432
}
428433

434+
@Override
435+
public boolean[] getBooleanArray(int index) {
436+
return getPrimitiveArray(schema.indexToName(index));
437+
}
438+
429439
@Override
430440
public Object[] getTuple(int index) {
431441
return readValue(index);

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
@@ -242,6 +242,8 @@ public interface GenericRecord {
242242
*/
243243
double[] getDoubleArray(String colName);
244244

245+
boolean[] getBooleanArray(String colName);
246+
245247
/**
246248
* Reads column with name `colName` as a string.
247249
*
@@ -469,6 +471,8 @@ public interface GenericRecord {
469471
*/
470472
double[] getDoubleArray(int index);
471473

474+
boolean[] getBooleanArray(int index);
475+
472476
Object[] getTuple(int index);
473477

474478
Object[] getTuple(String colName);

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

Lines changed: 61 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@
1212
import com.clickhouse.client.api.command.CommandResponse;
1313
import com.clickhouse.client.api.command.CommandSettings;
1414
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
15+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
1516
import com.clickhouse.client.api.enums.Protocol;
1617
import com.clickhouse.client.api.insert.InsertResponse;
1718
import com.clickhouse.client.api.insert.InsertSettings;
@@ -170,6 +171,8 @@ public void testReadRecords() throws Exception {
170171
Assert.assertEquals(record.getString("col3"), dsRecords.get("col3"));
171172
Assert.assertEquals(record.getLong("col4"), dsRecords.get("col4"));
172173
Assert.assertEquals(record.getString("col5"), dsRecords.get("col5"));
174+
Assert.assertEquals(record.getBooleanArray("col6"), ((List)dsRecords.get("col6")).toArray());
175+
Assert.assertEquals(record.getIntArray("col7"), ((List)dsRecords.get("col7")).toArray());
173176
}
174177
}
175178

@@ -286,7 +289,16 @@ public void testQueryAll() throws Exception {
286289
for (String colDefinition : DATASET_COLUMNS) {
287290
// result values
288291
String colName = colDefinition.split(" ")[0];
289-
List<Object> colValues = records.stream().map(r -> r.getObject(colName)).collect(Collectors.toList());
292+
List<Object> colValues = records.stream().map(r -> {
293+
Object v = r.getObject(colName);
294+
if (v instanceof BinaryStreamReader.ArrayValue) {
295+
v = ((BinaryStreamReader.ArrayValue)v).asList();
296+
}
297+
298+
return v;
299+
}
300+
301+
).collect(Collectors.toList());
290302
Assert.assertEquals(colValues.size(), dataset.size());
291303

292304
// dataset values
@@ -387,13 +399,36 @@ public void testRowBinaryQueries(ClickHouseFormat format)
387399
while (dataIterator.hasNext()) {
388400
Map<String, Object> expectedRecord = dataIterator.next();
389401
Map<String, Object> actualRecord = reader.next();
390-
Assert.assertEquals(actualRecord, expectedRecord);
402+
for (Map.Entry<String, Object> entry : actualRecord.entrySet()) {
403+
Object value = entry.getValue();
404+
if (entry.getValue() instanceof BinaryStreamReader.ArrayValue) {
405+
value = ((BinaryStreamReader.ArrayValue)value).asList();
406+
}
407+
408+
Assert.assertEquals(value, expectedRecord.get(entry.getKey()), "Value of " + entry.getKey() + " doesn't match: "
409+
+ expectedRecord.get(entry.getKey()) + " expected, actual: " + value);
410+
411+
}
391412
rowsCount++;
392413
}
393414

394415
Assert.assertEquals(rowsCount, rows);
395416
}
396417

418+
@Test
419+
public void testReadingArrayInNative() throws Exception {
420+
421+
QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes);
422+
try (QueryResponse response = client.query("SELECT [1, 2, 3] as arr1, [[1, 2, 3], [4, 5, 6]] as arr2", querySettings).get()) {
423+
ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response);
424+
425+
Map<String, Object> record = reader.next();
426+
Assert.assertEquals(((BinaryStreamReader.ArrayValue)record.get("arr1")).asList(), Arrays.asList((short)1, (short)2, (short)3));
427+
Assert.assertEquals(((BinaryStreamReader.ArrayValue)record.get("arr2")).asList().get(0), Arrays.asList((short)1, (short)2, (short)3));
428+
Assert.assertEquals(((BinaryStreamReader.ArrayValue)record.get("arr2")).asList().get(1), Arrays.asList((short)4, (short)5, (short)6));
429+
}
430+
}
431+
397432
@Test(groups = {"integration"})
398433
public void testBinaryStreamReader() throws Exception {
399434
final String table = "dynamic_schema_test_table";
@@ -450,7 +485,9 @@ record = reader.next();
450485

451486
private final static List<String> ARRAY_COLUMNS = Arrays.asList(
452487
"col1 Array(UInt32)",
453-
"col2 Array(Array(Int32))"
488+
"col2 Array(Array(Int32))",
489+
"col3 Array(UInt64)",
490+
"col4 Array(Bool)"
454491
);
455492

456493
private final static List<Function<String, Object>> ARRAY_VALUE_GENERATORS = Arrays.asList(
@@ -459,11 +496,18 @@ record = reader.next();
459496
.asLongStream().collect(ArrayList::new, ArrayList::add, ArrayList::addAll),
460497
c -> {
461498
List<List<Integer>> values = new ArrayList<>();
499+
462500
for (int i = 0; i < 10; i++) {
463-
values.add(Arrays.asList(1, 2, 3));
501+
values.add(Arrays.asList(i, 2 * i , 3 * i));
464502
}
465503
return values;
466-
}
504+
},
505+
c ->
506+
RANDOM.longs(10, 0, Long.MAX_VALUE)
507+
.mapToObj(BigInteger::valueOf).collect(Collectors.toList()),
508+
c -> RANDOM.ints(10, 0, 1)
509+
.mapToObj(i -> i == 0 ).collect(Collectors.toList())
510+
467511
);
468512

469513
@Test(groups = {"integration"})
@@ -497,6 +541,12 @@ public void testArrayValues() throws Exception {
497541
Assert.assertEquals(reader.getList("col1"), datasetRecord.get("col1"));
498542
List<List<Long>> col2Values = reader.getList("col2");
499543
Assert.assertEquals(col2Values, data.get(0).get("col2"));
544+
List<BigInteger> col3Values = reader.getList("col3");
545+
Assert.assertEquals(col3Values, data.get(0).get("col3"));
546+
List<Boolean> col4Values = reader.getList("col4");
547+
Assert.assertEquals(col4Values, data.get(0).get("col4"));
548+
boolean[] col4Array = reader.getBooleanArray("col4");
549+
Assert.assertEquals(col4Array, ((List)data.get(0).get("col4")).toArray());
500550
}
501551

502552
private final static List<String> MAP_COLUMNS = Arrays.asList(
@@ -1292,15 +1342,19 @@ public void testQueryMetrics() throws Exception {
12921342
"col2 Int32",
12931343
"col3 String",
12941344
"col4 Int64",
1295-
"col5 String"
1345+
"col5 String",
1346+
"col6 Array(Bool)",
1347+
"col7 Array(Int32)"
12961348
);
12971349

12981350
private final static List<Function<String, Object>> DATASET_VALUE_GENERATORS = Arrays.asList(
12991351
c -> Long.valueOf(RANDOM.nextInt(Integer.MAX_VALUE)),
13001352
c -> RANDOM.nextInt(Integer.MAX_VALUE),
13011353
c -> "value_" + RANDOM.nextInt(Integer.MAX_VALUE),
13021354
c -> Long.valueOf(RANDOM.nextInt(Integer.MAX_VALUE)),
1303-
c -> "value_" + RANDOM.nextInt(Integer.MAX_VALUE)
1355+
c -> "value_" + RANDOM.nextInt(Integer.MAX_VALUE),
1356+
c -> RANDOM.ints(10, 0, 1).mapToObj(i -> i == 0).collect(Collectors.toList()),
1357+
c -> RANDOM.ints(10, 0, Integer.MAX_VALUE).boxed().collect(Collectors.toList())
13041358
);
13051359

13061360
private final static String DATASET_TABLE = "query_test_table";

0 commit comments

Comments
 (0)