Skip to content

Commit e4e8402

Browse files
committed
Fixed reading JSON with array of objects
1 parent 0ac5de1 commit e4e8402

File tree

4 files changed

+182
-59
lines changed

4 files changed

+182
-59
lines changed

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -372,11 +372,11 @@ public byte getTag() {
372372
allAliases = Collections.unmodifiableSet(set);
373373
name2type = Collections.unmodifiableMap(map);
374374

375-
Map<Byte, ClickHouseDataType> tmpbinTag2Type = new HashMap<>();
375+
Map<Byte, ClickHouseDataType> tmpBinTag2Type = new HashMap<>();
376376
for (ClickHouseDataType type : ClickHouseDataType.values()) {
377-
tmpbinTag2Type.put((byte) type.getBinTag(), type);
377+
tmpBinTag2Type.put(type.getBinTag(), type);
378378
}
379-
binTag2Type = Collections.unmodifiableMap(tmpbinTag2Type);
379+
binTag2Type = Collections.unmodifiableMap(tmpBinTag2Type);
380380

381381
Map<Byte, ClickHouseDataType> tmpIntervalKind2Type = new HashMap<>();
382382
Map<ClickHouseDataType, ClickHouseDataType.IntervalKind > tmpIntervalType2Kind = new HashMap<>();

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

Lines changed: 163 additions & 52 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import java.util.Set;
3737
import java.util.TimeZone;
3838
import java.util.UUID;
39+
import java.util.stream.Collectors;
3940

4041
/**
4142
* This class is not thread safe and should not be shared between multiple threads.
@@ -1160,62 +1161,172 @@ public byte[] allocate(int size) {
11601161
private ClickHouseColumn readDynamicData() throws IOException {
11611162
byte tag = readByte();
11621163

1163-
ClickHouseDataType type;
1164-
if (tag == ClickHouseDataType.INTERVAL_BIN_TAG) {
1165-
byte intervalKind = readByte();
1166-
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
1167-
if (type == null) {
1168-
throw new ClientException("Unsupported interval kind: " + intervalKind);
1164+
ClickHouseDataType type = ClickHouseDataType.binTag2Type.get(tag);
1165+
if (type == null) {
1166+
throw new ClientException("Unsupported data type with tag " + tag);
1167+
}
1168+
switch (type) {
1169+
case Array: {
1170+
ClickHouseColumn elementColumn = readDynamicData();
1171+
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
11691172
}
1170-
return ClickHouseColumn.of("v", type, false, 0, 0);
1171-
} else if (tag == ClickHouseDataType.DateTime32.getBinTag()) {
1172-
byte scale = readByte();
1173-
return ClickHouseColumn.of("v", "DateTime32(" + scale + ")");
1174-
} else if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) { // without timezone
1175-
byte scale = readByte();
1176-
return ClickHouseColumn.of("v", "DateTime64(" + scale +")");
1177-
} else if (tag == ClickHouseDataType.DateTime64.getBinTag()) {
1178-
byte scale = readByte();
1179-
String timezone = readString(input);
1180-
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
1181-
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1182-
String typeName = readString(input);
1183-
return ClickHouseColumn.of("v", typeName);
1184-
} else if (DECIMAL_TAGS.contains(tag)) {
1185-
int precision = readByte();
1186-
int scale = readByte();
1187-
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
1188-
} else if (tag == ClickHouseDataType.Array.getBinTag()) {
1189-
ClickHouseColumn elementColumn = readDynamicData();
1190-
return ClickHouseColumn.of("v", "Array(" + elementColumn.getOriginalTypeName() + ")");
1191-
} else if (tag == ClickHouseDataType.Map.getBinTag()) {
1192-
ClickHouseColumn keyInfo = readDynamicData();
1193-
ClickHouseColumn valueInfo = readDynamicData();
1194-
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
1195-
} else if (tag == ClickHouseDataType.Enum8.getBinTag() || tag == ClickHouseDataType.Enum16.getBinTag()) {
1196-
int constants = readVarInt(input);
1197-
int[] values = new int[constants];
1198-
String[] names = new String[constants];
1199-
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
1200-
for (int i = 0; i < constants; i++) {
1201-
names[i] = readString(input);
1202-
if (enumType == ClickHouseDataType.Enum8) {
1203-
values[i] = readUnsignedByte();
1204-
} else {
1205-
values[i] = readUnsignedShortLE();
1173+
case DateTime32: {
1174+
String timezone = readString(input);
1175+
return ClickHouseColumn.of("v", "DateTime32(" + timezone + ")");
1176+
}
1177+
case DateTime64: {
1178+
byte scale = readByte();
1179+
String timezone = readString(input);
1180+
return ClickHouseColumn.of("v", "DateTime64(" + scale + (timezone.isEmpty() ? "" : ", " + timezone) +")");
1181+
}
1182+
case Decimal:
1183+
case Decimal32:
1184+
case Decimal64:
1185+
case Decimal128:
1186+
case Decimal256: {
1187+
int precision = readByte();
1188+
int scale = readByte();
1189+
return ClickHouseColumn.of("v", ClickHouseDataType.binTag2Type.get(tag), false, precision, scale);
1190+
}
1191+
case Dynamic: {
1192+
int maxTypes = readVarInt(input);
1193+
return ClickHouseColumn.of("v", "Dynamic(" + maxTypes + ")");
1194+
}
1195+
case Enum:
1196+
case Enum8:
1197+
case Enum16: {
1198+
int constants = readVarInt(input);
1199+
int[] values = new int[constants];
1200+
String[] names = new String[constants];
1201+
ClickHouseDataType enumType = constants > 127 ? ClickHouseDataType.Enum16 : ClickHouseDataType.Enum8;
1202+
for (int i = 0; i < constants; i++) {
1203+
names[i] = readString(input);
1204+
if (enumType == ClickHouseDataType.Enum8) {
1205+
values[i] = readUnsignedByte();
1206+
} else {
1207+
values[i] = readUnsignedShortLE();
1208+
}
12061209
}
1210+
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
1211+
new ClickHouseEnum(names, values));
12071212
}
1208-
return new ClickHouseColumn(enumType, "v", enumType.name(), false, false, Collections.emptyList(), Collections.emptyList(),
1209-
new ClickHouseEnum(names, values));
1210-
} else if (tag == ClickHouseDataType.NULLABLE_BIN_TAG) {
1211-
ClickHouseColumn column = readDynamicData();
1212-
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
1213-
} else {
1214-
type = ClickHouseDataType.binTag2Type.get(tag);
1215-
if (type == null) {
1216-
throw new ClientException("Unsupported data type with tag " + tag);
1213+
case FixedString: {
1214+
int length = readVarInt(input);
1215+
return ClickHouseColumn.of("v", "FixedString(" + length + ")");
12171216
}
1218-
return ClickHouseColumn.of("v", type, false, 0, 0);
1217+
case IntervalHour:
1218+
case IntervalMinute:
1219+
case IntervalSecond:
1220+
case IntervalDay:
1221+
case IntervalMonth:
1222+
case IntervalMicrosecond:
1223+
case IntervalMillisecond:
1224+
case IntervalNanosecond:
1225+
case IntervalQuarter:
1226+
case IntervalYear:
1227+
case IntervalWeek: {
1228+
byte intervalKind = readByte();
1229+
type = ClickHouseDataType.intervalKind2Type.get(intervalKind);
1230+
if (type == null) {
1231+
throw new ClientException("Unsupported interval kind: " + intervalKind);
1232+
}
1233+
return ClickHouseColumn.of("v", type, false, 0, 0);
1234+
}
1235+
case JSON: {
1236+
byte serializationVersion = readByte();
1237+
int maxDynamicPaths = readVarInt(input);
1238+
byte maxDynamicTypes = readByte();
1239+
int numberOfTypedPaths = readVarInt(input);
1240+
StringBuilder typeDef = new StringBuilder();
1241+
typeDef.append("JSON(max_dynamic_paths=").append(maxDynamicPaths).append(",max_dynamic_types=").append(maxDynamicTypes).append(",");
1242+
for (int i = 0; i < numberOfTypedPaths; i++) {
1243+
typeDef.append(readString(input)); // path
1244+
ClickHouseColumn column = readDynamicData();
1245+
typeDef.append(column.getOriginalTypeName()).append(',');
1246+
}
1247+
int numberOfSkipPaths = readVarInt(input);
1248+
for (int i = 0; i < numberOfSkipPaths; i++) {
1249+
typeDef.append(readString(input)).append(',');
1250+
}
1251+
int numberOfPathRegexp = readVarInt(input);
1252+
for (int i = 0; i < numberOfPathRegexp; i++) {
1253+
typeDef.append(readString(input)).append(',');
1254+
}
1255+
typeDef.setLength(typeDef.length() - 1);
1256+
typeDef.append(')');
1257+
return ClickHouseColumn.of("v", typeDef.toString());
1258+
}
1259+
case LowCardinality: {
1260+
ClickHouseColumn column = readDynamicData();
1261+
return ClickHouseColumn.of("v", "LowCardinality(" + column.getOriginalTypeName() + ")");
1262+
}
1263+
case Map: {
1264+
ClickHouseColumn keyInfo = readDynamicData();
1265+
ClickHouseColumn valueInfo = readDynamicData();
1266+
return ClickHouseColumn.of("v", "Map(" + keyInfo.getOriginalTypeName() + "," + valueInfo.getOriginalTypeName() + ")");
1267+
}
1268+
case Nested: {
1269+
int size = readVarInt(input);
1270+
StringBuilder nested = new StringBuilder();
1271+
nested.append("Nested(");
1272+
for (int i = 0; i < size; i++) {
1273+
String name = readString(input);
1274+
nested.append(name).append(',');
1275+
}
1276+
nested.setLength(nested.length() - 1);
1277+
nested.append(')');
1278+
return ClickHouseColumn.of("v", nested.toString());
1279+
}
1280+
case Nullable: {
1281+
ClickHouseColumn column = readDynamicData();
1282+
return ClickHouseColumn.of("v", "Nullable(" + column.getOriginalTypeName() + ")");
1283+
}
1284+
case Time64: {
1285+
byte precision = readByte();
1286+
return ClickHouseColumn.of("v", "Time64(" + precision + ")");
1287+
}
1288+
case Variant: {
1289+
int variants = readVarInt(input);
1290+
StringBuilder variant = new StringBuilder();
1291+
variant.append("Variant(");
1292+
for (int i = 0; i < variants; i++) {
1293+
ClickHouseColumn column = readDynamicData();
1294+
variant.append(column.getOriginalTypeName() + ",");
1295+
}
1296+
variant.setLength(variant.length() - 1);
1297+
variant.append(")");
1298+
return ClickHouseColumn.of("v", "Variant(" + variant + ")");
1299+
}
1300+
case AggregateFunction:
1301+
throw new ClientException("Aggregate functions are not supported yet");
1302+
case BFloat16:
1303+
throw new ClientException("BFloat16 is not supported yet");
1304+
default:
1305+
if (tag == ClickHouseDataType.DateTime64.getBinTag() - 1) {
1306+
// without timezone
1307+
byte scale = readByte();
1308+
return ClickHouseColumn.of("v", "DateTime64(" + scale + ")");
1309+
} else if (tag == ClickHouseDataType.CUSTOM_TYPE_BIN_TAG) {
1310+
String typeName = readString(input);
1311+
return ClickHouseColumn.of("v", typeName);
1312+
} else if (tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG || tag == ClickHouseDataType.TUPLE_WITHOUT_NAMES_BIN_TAG) {
1313+
int size = readVarInt(input);
1314+
StringBuilder typeNameBuilder = new StringBuilder();
1315+
typeNameBuilder.append("Tuple(");
1316+
final boolean readName = tag == ClickHouseDataType.TUPLE_WITH_NAMES_BIN_TAG;
1317+
for (int i = 0; i < size; i++) {
1318+
if (readName) {
1319+
String name = readString(input);
1320+
typeNameBuilder.append(name).append(' ');
1321+
}
1322+
ClickHouseColumn column = readDynamicData();
1323+
typeNameBuilder.append(column.getOriginalTypeName()).append(',');
1324+
}
1325+
typeNameBuilder.setLength(typeNameBuilder.length() - 1);
1326+
typeNameBuilder.append(")");
1327+
return ClickHouseColumn.of("v", typeNameBuilder.toString());
1328+
}
1329+
return ClickHouseColumn.of("v", type, false, 0, 0);
12191330
}
12201331
}
12211332

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -659,7 +659,7 @@ private static void serializeTime64(OutputStream stream, Object value) throws IO
659659
} else if (value instanceof Long) {
660660
BinaryStreamUtils.writeUnsignedInt64(stream, (Long) value);
661661
} else if (value instanceof Instant) {
662-
BinaryStreamUtils.writeUnsignedInt64(stream, BigInteger.valueOf(((Instant) value).getEpochSecond()).shiftLeft(32)
662+
BinaryStreamUtils.writeUnsignedInt64(stream,BigInteger.valueOf(((Instant) value).getEpochSecond() * 1_000_000_000L)
663663
.add(BigInteger.valueOf(((Instant) value).getNano())));
664664
} else {
665665
throw new UnsupportedOperationException("Cannot convert " + value.getClass() + " to Time64");

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

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -652,15 +652,27 @@ public void testDynamicWithTime64Types() throws Exception {
652652
String.valueOf(_999_hours),
653653
});
654654

655-
Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59,
656-
999999999);
655+
Instant time64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59);
656+
long time64Value = time64.getEpochSecond() * 1_000_000_000 + time64.getNano();
657+
System.out.println(time64Value);
658+
testDynamicWith("Time64",
659+
new Object[]{
660+
time64Value
661+
},
662+
new String[]{
663+
String.valueOf(time64Value)
664+
}
665+
);
657666

667+
Instant maxTime64 = Instant.ofEpochSecond(TimeUnit.HOURS.toSeconds(999) + TimeUnit.MINUTES.toSeconds(59) + 59,
668+
123456789);
669+
long maxTime64Value = maxTime64.getEpochSecond() * 1_000_000_000 + maxTime64.getNano();
658670
testDynamicWith("Time64",
659671
new Object[]{
660672
maxTime64,
661673
},
662674
new String[]{
663-
"3958241016481971977"
675+
String.valueOf(maxTime64Value)
664676
});
665677
}
666678

0 commit comments

Comments
 (0)