Skip to content

Commit 9c3a6ed

Browse files
committed
fixed array handling when Dynamic
1 parent d2b6d59 commit 9c3a6ed

File tree

5 files changed

+162
-50
lines changed

5 files changed

+162
-50
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -640,7 +640,7 @@ public static List<ClickHouseColumn> parse(String args) {
640640
return Collections.unmodifiableList(c);
641641
}
642642

643-
private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable,
643+
public ClickHouseColumn(ClickHouseDataType dataType, String columnName, String originalTypeName, boolean nullable,
644644
boolean lowCardinality, List<String> parameters, List<ClickHouseColumn> nestedColumns) {
645645
this.aggFuncType = null;
646646
this.dataType = ClickHouseChecker.nonNull(dataType, "dataType");

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

Lines changed: 35 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -21,11 +21,14 @@
2121
import java.time.LocalDateTime;
2222
import java.time.ZonedDateTime;
2323
import java.util.ArrayList;
24+
import java.util.Arrays;
2425
import java.util.Collections;
2526
import java.util.HashMap;
27+
import java.util.HashSet;
2628
import java.util.LinkedHashMap;
2729
import java.util.List;
2830
import java.util.Map;
31+
import java.util.Set;
2932
import java.util.Stack;
3033
import java.util.TimeZone;
3134
import java.util.UUID;
@@ -93,18 +96,12 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
9396
}
9497
}
9598

96-
ClickHouseDataType dataType = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column.getDataType();
97-
int estimatedLen = column.getEstimatedLength();
98-
int precision = column.getPrecision();
99-
int scale = column.getScale();
100-
if (dataType == ClickHouseDataType.Decimal || dataType == ClickHouseDataType.Decimal32 ||
101-
dataType == ClickHouseDataType.Decimal64 || dataType == ClickHouseDataType.Decimal128 ||
102-
dataType == ClickHouseDataType.Decimal256) {
103-
precision = readByte();
104-
scale = readByte();
105-
System.out.println("p: " + precision + " " +scale);
106-
}
107-
TimeZone timezone = column.getTimeZoneOrDefault(timeZone);
99+
ClickHouseColumn actualColumn = column.getDataType() == ClickHouseDataType.Dynamic ? readDynamicData() : column;
100+
ClickHouseDataType dataType = actualColumn.getDataType();
101+
int estimatedLen = actualColumn.getEstimatedLength();
102+
int precision = actualColumn.getPrecision();
103+
int scale = actualColumn.getScale();
104+
TimeZone timezone = actualColumn.getTimeZoneOrDefault(timeZone);
108105

109106
try {
110107
switch (dataType) {
@@ -211,24 +208,24 @@ public <T> T readValue(ClickHouseColumn column, Class<?> typeHint) throws IOExce
211208
}
212209
// case Object: // deprecated https://clickhouse.com/docs/en/sql-reference/data-types/object-data-type
213210
case Array:
214-
return convertArray(readArray(column), typeHint);
211+
return convertArray(readArray(actualColumn), typeHint);
215212
case Map:
216-
return (T) readMap(column);
213+
return (T) readMap(actualColumn);
217214
// case Nested:
218215
case Tuple:
219-
return (T) readTuple(column);
216+
return (T) readTuple(actualColumn);
220217
case Nothing:
221218
return null;
222219
case SimpleAggregateFunction:
223220
return (T) readValue(column.getNestedColumns().get(0));
224221
case AggregateFunction:
225-
return (T) readBitmap( column);
222+
return (T) readBitmap( actualColumn);
226223
case Variant:
227-
return (T) readVariant(column);
224+
return (T) readVariant(actualColumn);
228225
case Dynamic:
229-
return (T) readValue(column, typeHint);
226+
return (T) readValue(actualColumn, typeHint);
230227
default:
231-
throw new IllegalArgumentException("Unsupported data type: " + column.getDataType());
228+
throw new IllegalArgumentException("Unsupported data type: " + actualColumn.getDataType());
232229
}
233230
} catch (EOFException e) {
234231
throw e;
@@ -1003,7 +1000,15 @@ public byte[] allocate(int size) {
10031000
}
10041001
}
10051002

1006-
private ClickHouseDataType readDynamicData() throws IOException {
1003+
private static final Set<Byte> DECIMAL_TAGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
1004+
ClickHouseDataType.Decimal.getBinTag(),
1005+
ClickHouseDataType.Decimal32.getBinTag(),
1006+
ClickHouseDataType.Decimal64.getBinTag(),
1007+
ClickHouseDataType.Decimal128.getBinTag(),
1008+
ClickHouseDataType.Decimal256.getBinTag()
1009+
)));
1010+
1011+
private ClickHouseColumn readDynamicData() throws IOException {
10071012
byte tag = readByte();
10081013

10091014
ClickHouseDataType type;
@@ -1013,25 +1018,24 @@ private ClickHouseDataType readDynamicData() throws IOException {
10131018
if (type == null) {
10141019
throw new ClientException("Unsupported interval kind: " + intervalKind);
10151020
}
1021+
return ClickHouseColumn.of("v", type, false, 0, 0);
10161022
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
10171023
String typeName = readString(input);
1018-
return ClickHouseDataType.valueOf(typeName);
1019-
} else if (tag == ClickHouseDataType.Decimal32.getBinTag()) {
1020-
return ClickHouseDataType.Decimal;
1021-
} else if (tag == ClickHouseDataType.Decimal64.getBinTag()) {
1022-
return ClickHouseDataType.Decimal;
1023-
} else if (tag == ClickHouseDataType.Decimal128.getBinTag()) {
1024-
return ClickHouseDataType.Decimal;
1025-
} else if (tag == ClickHouseDataType.Decimal256.getBinTag()) {
1026-
return ClickHouseDataType.Decimal;
1024+
return ClickHouseColumn.of("v", typeName);
1025+
} else if (DECIMAL_TAGS.contains(tag)) {
1026+
int precision = readByte();
1027+
int scale = readByte();
1028+
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
1029+
} else if (tag == ClickHouseDataType.Array.getBinTag()) {
1030+
ClickHouseColumn elementColumn = readDynamicData();
1031+
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
10271032
} else {
10281033
type = ClickHouseDataType.binTag2Type.get(tag);
10291034
if (type == null) {
10301035
throw new ClientException("Unsupported data type with tag " + tag);
10311036
}
1037+
return ClickHouseColumn.of("v", type, false, 0, 0);
10321038
}
1033-
1034-
return type;
10351039
}
10361040

10371041
private static final ClickHouseColumn JSON_PLACEHOLDER_COL = ClickHouseColumn.parse("v Dynamic").get(0);

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

Lines changed: 39 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,7 @@
4646
import java.util.TimeZone;
4747
import java.util.UUID;
4848
import java.util.stream.Collectors;
49+
import java.util.stream.StreamSupport;
4950

5051
import static org.objectweb.asm.Opcodes.ACC_PUBLIC;
5152
import static org.objectweb.asm.Opcodes.ALOAD;
@@ -95,7 +96,7 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo
9596
break;
9697
case Dynamic:
9798
ClickHouseColumn typeColumn = valueToColumnForDynamicType(value);
98-
writeDynamicTypeTag(stream, typeColumn, value);
99+
writeDynamicTypeTag(stream, typeColumn);
99100
serializeData(stream, value, typeColumn);
100101
break;
101102
default:
@@ -158,6 +159,10 @@ private static Map<Class<?>, ClickHouseColumn> getPredefinedTypeColumnsMap() {
158159
map.put(double[][].class, ClickHouseColumn.of("v", "Array(Array(Float64))"));
159160
map.put(double[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(Float64)))"));
160161

162+
map.put(String[].class, ClickHouseColumn.of("v", "Array(String)"));
163+
map.put(String[][].class, ClickHouseColumn.of("v", "Array(Array(String))"));
164+
map.put(String[][][].class, ClickHouseColumn.of("v", "Array(Array(Array(String)))"));
165+
161166
return Collections.unmodifiableMap(map);
162167
}
163168

@@ -187,40 +192,59 @@ public static ClickHouseColumn valueToColumnForDynamicType(Object value) {
187192
ClickHouseColumn valueInfo = valueToColumnForDynamicType(entry.getValue());
188193
column = ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + ", " + valueInfo.getOriginalTypeName() + ")");
189194
} else if (value instanceof List<?>) {
195+
column = value2Column(value);
196+
} else if (value == null) {
197+
column = PREDEFINED_TYPE_COLUMNS.get(Void.class);
198+
} else {
199+
column = PREDEFINED_TYPE_COLUMNS.get(value.getClass());
200+
}
201+
202+
if (column == null) {
203+
throw new ClientException("Unable to serialize value of " + value.getClass() + " because not supported yet");
204+
}
205+
206+
return column;
207+
}
208+
209+
// Returns null if cannot convert
210+
// The problem here is that >2-dimensional array would require traversing all value
211+
// to detect correct depth. Consider this example
212+
// int[][] {
213+
// null
214+
// { }
215+
// { 0, 1, 2 }
216+
// In this case we need to find max depth.
217+
218+
private static ClickHouseColumn value2Column(Object value) {
219+
ClickHouseColumn column;
220+
if (value instanceof List<?>) {
190221
List<?> list = (List<?>) value;
191222
StringBuilder type = new StringBuilder("Array()");
192-
int insertPos = type.length() - 2;
223+
int insertPos = type.length() - 1;
193224
while (!list.isEmpty() && list.get(0) instanceof List<?>) {
194225
type.insert(insertPos, "Array()");
195226
insertPos += 6; // add len of 'Array(' string
196227
list = (List<?>) list.get(0);
197228
}
198-
if (list.isEmpty()) {
229+
if (list.isEmpty() || list.get(0) == null) {
199230
type.insert(insertPos, "Nothing");
200231
column = ClickHouseColumn.of("v", type.toString());
201232
} else {
202-
ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0));
233+
ClickHouseColumn arrayBaseColumn = PREDEFINED_TYPE_COLUMNS.get(list.get(0).getClass());
203234
if (arrayBaseColumn != null) {
204235
type.insert(insertPos, arrayBaseColumn.getOriginalTypeName());
205236
column = ClickHouseColumn.of("v", type.toString());
206237
} else {
207238
column = null;
208239
}
209240
}
210-
} else if (value == null) {
211-
column = PREDEFINED_TYPE_COLUMNS.get(Void.class);
212241
} else {
213-
column = PREDEFINED_TYPE_COLUMNS.get(value.getClass());
214-
}
215-
216-
if (column == null) {
217-
throw new ClientException("Unable to serialize value of " + value.getClass() + " because not supported yet");
242+
column = null;
218243
}
219-
220244
return column;
221245
}
222246

223-
public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn, Object value)
247+
public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typeColumn)
224248
throws IOException {
225249

226250
ClickHouseDataType dt = typeColumn.getDataType();
@@ -285,7 +309,8 @@ public static void writeDynamicTypeTag(OutputStream stream, ClickHouseColumn typ
285309
break;
286310
case Array:
287311
stream.write(binTag);
288-
// elements 0x1E<nested_type_encoding>
312+
ClickHouseColumn arrayElemColumn = typeColumn.getNestedColumns().get(0);
313+
writeDynamicTypeTag(stream, arrayElemColumn);
289314
break;
290315
case Map:
291316
stream.write(binTag);

client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypeTests.java

Lines changed: 84 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -410,15 +410,35 @@ public void testDynamicWithPrimitives() throws Exception {
410410
int rowId = 0;
411411
for (ClickHouseDataType dataType : ClickHouseDataType.values()) {
412412
System.out.println("Testing dynamic with " + dataType + " values");
413-
414413
switch (dataType) {
415414
case Date:
416415
case Date32:
417416
case DateTime:
418417
case DateTime32:
419418
case DateTime64:
419+
case Decimal:
420+
case Decimal32:
421+
case Decimal64:
422+
case Decimal128:
423+
case Decimal256:
424+
// requires fix
425+
continue;
426+
case Array:
427+
case Map:
428+
case Nested:
429+
case Tuple:
430+
case AggregateFunction:
431+
case SimpleAggregateFunction:
432+
case Variant:
420433
case Enum8:
421434
case Enum16:
435+
// tested separately
436+
continue;
437+
case Dynamic:
438+
case Nothing:
439+
case Object:
440+
case JSON:
441+
// no tests or tested in other tests
422442
continue;
423443
default:
424444
}
@@ -480,13 +500,76 @@ public void testDynamicWithPrimitives() throws Exception {
480500
}
481501
}
482502

503+
@Test(groups = {"integration"})
504+
public void testDynamicWithArrays() throws Exception {
505+
testDynamicWith("arrays",
506+
new Object[]{
507+
"a,b",
508+
new String[]{"a", "b"},
509+
Arrays.asList("c", "d")
510+
},
511+
new String[]{
512+
"a,b",
513+
"[a, b]",
514+
"[c, d]"
515+
});
516+
testDynamicWith("arrays",
517+
new Object[]{
518+
new int[]{1, 2},
519+
new String[]{"a", "b"},
520+
Arrays.asList("c", "d")
521+
},
522+
new String[]{
523+
"[1, 2]",
524+
"[a, b]",
525+
"[c, d]",
526+
});
527+
528+
testDynamicWith("arrays",
529+
new Object[]{
530+
new int[][]{ new int[] {1, 2}, new int[] { 3, 4}},
531+
new String[][]{new String[]{"a", "b"}, new String[]{"c", "d"}},
532+
Arrays.asList(Arrays.asList("e", "f"), Arrays.asList("j", "h"))
533+
},
534+
new String[]{
535+
"[[1, 2], [3, 4]]",
536+
"[[a, b], [c, d]]",
537+
"[[e, f], [j, h]]",
538+
});
539+
}
540+
483541
@Data
484542
@AllArgsConstructor
485543
public static class DTOForDynamicPrimitivesTests {
486544
private int rowId;
487545
private Object field;
488546
}
489547

548+
private void testDynamicWith(String withWhat, Object[] values, String[] expectedStrValues) throws Exception {
549+
if (isVersionMatch("(,24.8]")) {
550+
return;
551+
}
552+
553+
String table = "test_dynamic_with_" + withWhat;
554+
client.execute("DROP TABLE IF EXISTS " + table).get();
555+
client.execute(tableDefinition(table, "rowId Int32", "field Dynamic"),
556+
(CommandSettings) new CommandSettings().serverSetting("enable_dynamic_type", "1")).get();
557+
558+
client.register(DTOForDynamicPrimitivesTests.class, client.getTableSchema(table));
559+
560+
List<DTOForDynamicPrimitivesTests> data = new ArrayList<>();
561+
for (int i = 0; i < values.length; i++) {
562+
data.add(new DTOForDynamicPrimitivesTests(i, values[i]));
563+
}
564+
client.insert(table, data).get().close();
565+
566+
List<GenericRecord> rows = client.queryAll("SELECT * FROM " + table);
567+
for (GenericRecord row : rows) {
568+
System.out.println("> " + row.getString("field"));
569+
Assert.assertEquals(row.getString("field"), expectedStrValues[row.getInteger("rowId")]);
570+
}
571+
}
572+
490573
private void testVariantWith(String withWhat, String[] fields, Object[] values, String[] expectedStrValues) throws Exception {
491574
if (isVersionMatch("(,24.8]")) {
492575
return;

client-v2/src/test/java/com/clickhouse/client/datatypes/DataTypesTestingPOJO.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -241,16 +241,16 @@ public DataTypesTestingPOJO() {
241241
groupBitmapUint32 = ClickHouseBitmap.wrap(random.ints(5, Integer.MAX_VALUE - 100, Integer.MAX_VALUE).toArray());
242242
groupBitmapUint64 = ClickHouseBitmap.wrap(random.longs(5, Long.MAX_VALUE - 100, Long.MAX_VALUE).toArray());
243243

244-
intervalYear = random.nextInt(2000, 4000);
244+
intervalYear = random.nextInt(4000);
245245
intervalQuarter = (byte) random.nextInt(4);
246246
intervalMonth = (byte) random.nextInt(12);
247247
intervalWeek = (byte) random.nextInt(52);
248248
intervalDay = (byte) random.nextInt(30);
249249
intervalHour = (byte) random.nextInt(24);
250250
intervalMinute = (byte) random.nextInt(60);
251251
intervalSecond = (byte) random.nextInt(60);
252-
intervalMillisecond = random.nextLong(10000);
253-
intervalMicrosecond = random.nextLong(10000);
252+
intervalMillisecond = random.nextLong();
253+
intervalMicrosecond = random.nextLong();
254254

255255
upper = BigInteger.valueOf(random.nextLong()).shiftLeft(64);
256256
lower = BigInteger.valueOf(random.nextLong()).and(BigInteger.valueOf(Long.MAX_VALUE));

0 commit comments

Comments
 (0)