Skip to content

Commit a18bf21

Browse files
author
Paultagoras
committed
Fixed tuple support
1 parent 5fecb57 commit a18bf21

File tree

3 files changed

+32
-28
lines changed

3 files changed

+32
-28
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -188,7 +188,7 @@ public void register(Class<?> clazz, TableSchema schema) {
188188
List<POJOSerializer> serializers = new ArrayList<>();
189189
Map<String, Method> getterMethods = new HashMap<>();
190190

191-
for (Method method: clazz.getMethods()) {
191+
for (Method method: clazz.getMethods()) {//Clean up the method names
192192
String methodName = method.getName();
193193
if (methodName.startsWith("get") || methodName.startsWith("has")) {
194194
methodName = methodName.substring(3).toLowerCase();
@@ -198,7 +198,7 @@ public void register(Class<?> clazz, TableSchema schema) {
198198
getterMethods.put(methodName, method);
199199
}
200200
}
201-
this.getterMethods.put(clazz, getterMethods);
201+
this.getterMethods.put(clazz, getterMethods);//Store the getter methods for later use
202202

203203
for (ClickHouseColumn column : schema.getColumns()) {
204204
String columnName = column.getColumnName().toLowerCase().replace("_", "");
@@ -222,7 +222,6 @@ public void register(Class<?> clazz, TableSchema schema) {
222222

223223
//Handle the different types
224224
SerializerUtils.serializeData(stream, value, column);
225-
226225
});
227226
}
228227
this.serializers.put(clazz, serializers);

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

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,11 +21,14 @@
2121

2222
public class SerializerUtils {
2323
public static void serializeData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException {
24+
//Serialize the value to the stream based on the data type
2425
switch (column.getDataType()) {
2526
case Array:
26-
case Tuple:
2727
serializeArrayData(stream, value, column);
2828
break;
29+
case Tuple:
30+
serializeTupleData(stream, value, column);
31+
break;
2932
case Map:
3033
serializeMapData(stream, value, column);
3134
break;
@@ -46,6 +49,15 @@ private static void serializeArrayData(OutputStream stream, Object value, ClickH
4649
}
4750
}
4851

52+
private static void serializeTupleData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException {
53+
//Serialize the tuple to the stream
54+
//The tuple is a list of values
55+
List<?> values = (List<?>) value;
56+
for (int i = 0; i < values.size(); i++) {
57+
serializeData(stream, values.get(i), column.getNestedColumns().get(i));
58+
}
59+
}
60+
4961
private static void serializeMapData(OutputStream stream, Object value, ClickHouseColumn column) throws IOException {
5062
//Serialize the map to the stream
5163
//The map is a list of key-value pairs

client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java

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

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.data.ClickHouseEnum;
5+
import org.apache.commons.lang3.RandomStringUtils;
46

57
import java.math.BigDecimal;
68
import java.math.BigInteger;
@@ -64,7 +66,7 @@ public class SamplePOJO {
6466

6567
public SamplePOJO() {
6668
int8 = RandomGenerator.getDefault().nextInt(-127, 128);
67-
int16 = RandomGenerator.getDefault().nextInt(-32768,32768);
69+
int16 = RandomGenerator.getDefault().nextInt(-32767,32768);
6870
int32 = RandomGenerator.getDefault().nextInt();
6971
int64 = RandomGenerator.getDefault().nextLong(9223372036854775807L);
7072
int128 = BigInteger.valueOf(RandomGenerator.getDefault().nextLong());
@@ -74,8 +76,8 @@ public SamplePOJO() {
7476
uint16 = RandomGenerator.getDefault().nextInt(0, 65536);
7577
uint32 = RandomGenerator.getDefault().nextLong(0, 4294967296L);
7678
uint64 = RandomGenerator.getDefault().nextLong(0, Long.MAX_VALUE);
77-
uint128 = BigInteger.valueOf(RandomGenerator.getDefault().nextLong());
78-
uint256 = BigInteger.valueOf(RandomGenerator.getDefault().nextLong());
79+
uint128 = BigInteger.valueOf(RandomGenerator.getDefault().nextLong(0, Long.MAX_VALUE));
80+
uint256 = BigInteger.valueOf(RandomGenerator.getDefault().nextLong(0, Long.MAX_VALUE));
7981

8082
float32 = RandomGenerator.getDefault().nextFloat();
8183
float64 = RandomGenerator.getDefault().nextDouble();
@@ -87,8 +89,8 @@ public SamplePOJO() {
8789

8890
bool = RandomGenerator.getDefault().nextBoolean();
8991

90-
string = "string" + RandomGenerator.getDefault().nextInt();
91-
fixedString = "fixed" + RandomGenerator.getDefault().nextInt();
92+
string = RandomStringUtils.randomAlphabetic(1, 512);
93+
fixedString = RandomStringUtils.randomAlphabetic(3);
9294

9395
date = LocalDate.now();
9496
date32 = LocalDate.now();
@@ -98,8 +100,8 @@ public SamplePOJO() {
98100

99101
uuid = UUID.randomUUID();
100102

101-
enum8 = (byte) RandomGenerator.getDefault().nextInt();
102-
enum16 = RandomGenerator.getDefault().nextInt();
103+
enum8 = (byte) RandomGenerator.getDefault().nextInt(0, 27);
104+
enum16 = RandomGenerator.getDefault().nextInt(0, 27);
103105

104106
try {
105107
Random random = new Random();
@@ -115,9 +117,9 @@ public SamplePOJO() {
115117
ipv6 = null;
116118
}
117119

118-
array = List.of("a", "b", "c");
119-
tuple = List.of(1, 2, 3);
120-
map = Map.of("a", 1, "b", 2, "c", 3);
120+
array = List.of("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", "t", "u", "v", "w", "x", "y", "z");
121+
tuple = List.of(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
122+
map = Map.of("a", 1, "b", 2, "c", 3, "d", 4, "e", 5, "f", 6, "g", 7, "h", 8, "i", 9, "j", 10);
121123

122124
inner = new SamplePOJOInner();
123125
}
@@ -423,7 +425,7 @@ public static TableSchema generateTableSchema(String tableName) {
423425
schema.addColumn("bool", "UInt8");
424426

425427
schema.addColumn("string", "String");
426-
schema.addColumn("fixedString", "FixedString(10)");
428+
schema.addColumn("fixedString", "FixedString(3)");
427429

428430
schema.addColumn("date", "Date");
429431
schema.addColumn("date32", "Date");
@@ -433,14 +435,14 @@ public static TableSchema generateTableSchema(String tableName) {
433435

434436
schema.addColumn("uuid", "UUID");
435437

436-
schema.addColumn("enum8", "Enum8('a' = 1, 'b' = 2, 'c' = 3)");
437-
schema.addColumn("enum16", "Enum16('a' = 1, 'b' = 2, 'c' = 3)");
438+
schema.addColumn("enum8", "Enum8('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26)");
439+
schema.addColumn("enum16", "Enum16('a' = 1, 'b' = 2, 'c' = 3, 'd' = 4, 'e' = 5, 'f' = 6, 'g' = 7, 'h' = 8, 'i' = 9, 'j' = 10, 'k' = 11, 'l' = 12, 'm' = 13, 'n' = 14, 'o' = 15, 'p' = 16, 'q' = 17, 'r' = 18, 's' = 19, 't' = 20, 'u' = 21, 'v' = 22, 'w' = 23, 'x' = 24, 'y' = 25, 'z' = 26)");
438440

439441
schema.addColumn("ipv4", "IPv4");
440442
schema.addColumn("ipv6", "IPv6");
441443

442444
schema.addColumn("array", "Array(String)");
443-
schema.addColumn("tuple", "Tuple(Int32, Int32, Int32)");
445+
schema.addColumn("tuple", "Tuple(Int32, Int32, Int32, Int32, Int32, Int32, Int32, Int32, Int32, Int32)");
444446
schema.addColumn("map", "Map(String, Int32)");
445447

446448
return schema;
@@ -454,16 +456,7 @@ public static String generateTableCreateSQL(String tableName) {
454456
if (i > 0) {
455457
sb.append(", ");
456458
}
457-
sb.append(schema.getColumns().get(i).getColumnName()).append(" ").append(schema.getColumns().get(i).getDataType());
458-
String type = String.valueOf(schema.getColumns().get(i).getDataType());
459-
460-
if (type.startsWith("Decimal")) {
461-
sb.append("(").append(schema.getColumns().get(i).getScale()).append(")");
462-
}
463-
464-
if (type.startsWith("FixedString")) {
465-
sb.append("(").append(schema.getColumns().get(i).getPrecision()).append(")");
466-
}
459+
sb.append(schema.getColumns().get(i).getColumnName()).append(" ").append(schema.getColumns().get(i).getOriginalTypeName());
467460
}
468461
sb.append(") ENGINE = Memory");
469462
return sb.toString();

0 commit comments

Comments
 (0)