Skip to content

Commit 9e03ba4

Browse files
committed
compiled serializers
1 parent 4031be1 commit 9e03ba4

File tree

8 files changed

+857
-115
lines changed

8 files changed

+857
-115
lines changed

client-v2/pom.xml

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,14 @@
8585
<scope>provided</scope>
8686
</dependency>
8787

88+
<!-- https://mvnrepository.com/artifact/org.ow2.asm/asm -->
89+
<dependency>
90+
<groupId>org.ow2.asm</groupId>
91+
<artifactId>asm</artifactId>
92+
<version>9.7</version>
93+
</dependency>
94+
95+
8896
<!-- Test dependencies -->
8997
<dependency>
9098
<groupId>${project.parent.groupId}</groupId>

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

Lines changed: 5 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@
3232
import com.clickhouse.client.api.metrics.ClientMetrics;
3333
import com.clickhouse.client.api.metrics.OperationMetrics;
3434
import com.clickhouse.client.api.query.GenericRecord;
35-
import com.clickhouse.client.api.query.POJODeserializer;
35+
import com.clickhouse.client.api.query.POJOSetter;
3636
import com.clickhouse.client.api.query.QueryResponse;
3737
import com.clickhouse.client.api.query.QuerySettings;
3838
import com.clickhouse.client.api.query.Records;
@@ -126,7 +126,7 @@ public class Client implements AutoCloseable {
126126
private final Map<Class<?>, List<POJOSerializer>> serializers; //Order is important to preserve for RowBinary
127127
private final Map<Class<?>, Map<String, Method>> getterMethods;
128128

129-
private final Map<Class<?>, Map<String, POJODeserializer>> deserializers;
129+
private final Map<Class<?>, Map<String, POJOSetter>> deserializers;
130130
private final Map<Class<?>, Map<String, Method>> setterMethods;
131131

132132
private final Map<Class<?>, Boolean> hasDefaults; // Whether the POJO has defaults
@@ -938,7 +938,7 @@ public synchronized void register(Class<?> clazz, TableSchema schema) {
938938
this.hasDefaults.put(clazz, schema.hasDefaults());
939939

940940
List<POJOSerializer> classSerializers = new ArrayList<>();
941-
Map<String, POJODeserializer> classDeserializers = new ConcurrentHashMap<>();
941+
Map<String, POJOSetter> classDeserializers = new ConcurrentHashMap<>();
942942
for (ClickHouseColumn column : schema.getColumns()) {
943943
String propertyName = column.getColumnName().toLowerCase().replace("_", "").replace(".", "");
944944

@@ -977,34 +977,7 @@ public synchronized void register(Class<?> clazz, TableSchema schema) {
977977
Method setterMethod = classSetters.get(propertyName);
978978
String columnName = column.getColumnName();
979979
if (setterMethod != null) {
980-
Class<?> argType = setterMethod.getParameterTypes()[0];
981-
if (argType.isPrimitive()) {
982-
if (argType.getName().equalsIgnoreCase("boolean")) {
983-
classDeserializers.put(columnName, SerializerUtils.booleanDeserializer(setterMethod));
984-
} else if (argType.getName().equalsIgnoreCase("byte")) {
985-
classDeserializers.put(columnName, SerializerUtils.byteDeserializer(setterMethod));
986-
} else if (argType.getName().equalsIgnoreCase("short")) {
987-
classDeserializers.put(columnName, SerializerUtils.shortDeserializer(setterMethod));
988-
} else if (argType.getName().equalsIgnoreCase("int")) {
989-
classDeserializers.put(columnName, SerializerUtils.intDeserializer(setterMethod));
990-
} else if (argType.getName().equalsIgnoreCase("long")) {
991-
classDeserializers.put(columnName, SerializerUtils.longDeserializer(setterMethod));
992-
} else if (argType.getName().equalsIgnoreCase("float")) {
993-
classDeserializers.put(columnName, SerializerUtils.floatDeserializer(setterMethod));
994-
} else if (argType.getName().equalsIgnoreCase("double")) {
995-
classDeserializers.put(columnName, SerializerUtils.doubleDeserializer(setterMethod));
996-
} else {
997-
throw new IllegalArgumentException("Unsupported primitive type: " + argType.getName() + " " + argType);
998-
}
999-
} else if (argType.isAssignableFrom(LocalDateTime.class)) {
1000-
classDeserializers.put(columnName, SerializerUtils.localDateTimeDeserializer(setterMethod));
1001-
} else if (argType.isAssignableFrom(LocalDate.class)) {
1002-
classDeserializers.put(columnName, SerializerUtils.localDateDeserializer(setterMethod));
1003-
} else if (argType.isAssignableFrom(List.class)) {
1004-
classDeserializers.put(columnName, SerializerUtils.listDeserializer(setterMethod));
1005-
} else {
1006-
classDeserializers.put(columnName, SerializerUtils.defaultPOJODeserializer(setterMethod));
1007-
}
980+
classDeserializers.put(columnName, SerializerUtils.compilePOJOSetter(setterMethod));
1008981
} else {
1009982
LOG.warn("No setter method found for column: {}", propertyName);
1010983
}
@@ -1554,7 +1527,7 @@ public <T> List<T> queryAll(String sqlQuery, Class<T> clazz) {
15541527
* @param <T>
15551528
*/
15561529
public <T> List<T> queryAll(String sqlQuery, Class<T> clazz, Supplier<T> allocator) {
1557-
Map<String, POJODeserializer> classDeserializers = deserializers.get(clazz);
1530+
Map<String, POJOSetter> classDeserializers = deserializers.get(clazz);
15581531

15591532
if (classDeserializers == null || classDeserializers.isEmpty()) {
15601533
throw new IllegalArgumentException("No deserializers found for class '" + clazz + "'. Did you forget to register it?");

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
55
import com.clickhouse.client.api.metadata.TableSchema;
66
import com.clickhouse.client.api.query.NullValueException;
7-
import com.clickhouse.client.api.query.POJODeserializer;
7+
import com.clickhouse.client.api.query.POJOSetter;
88
import com.clickhouse.client.api.query.QuerySettings;
99
import com.clickhouse.client.config.ClickHouseClientOption;
1010
import com.clickhouse.data.ClickHouseColumn;
@@ -73,16 +73,16 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer
7373

7474
protected AtomicBoolean nextRecordEmpty = new AtomicBoolean(true);
7575

76-
public boolean readToPOJO(Map<String, POJODeserializer> deserializers, Object obj ) throws IOException {
76+
public boolean readToPOJO(Map<String, POJOSetter> deserializers, Object obj ) throws IOException {
7777
boolean firstColumn = true;
7878

7979
for (ClickHouseColumn column : columns) {
8080
try {
8181
Object val = binaryStreamReader.readValue(column);
8282
if (val != null) {
83-
POJODeserializer deserializer = deserializers.get(column.getColumnName());
83+
POJOSetter deserializer = deserializers.get(column.getColumnName());
8484
if (deserializer != null) {
85-
deserializer.deserialize(obj, val);
85+
deserializer.setValue(obj, val);
8686
}
8787
}
8888
firstColumn = false;
@@ -92,7 +92,7 @@ public boolean readToPOJO(Map<String, POJODeserializer> deserializers, Object ob
9292
return false;
9393
}
9494
throw e;
95-
} catch (IllegalAccessException | InvocationTargetException e) {
95+
} catch (Exception e) {
9696
throw new ClientException("Failed to put value into POJO", e);
9797
}
9898
}

0 commit comments

Comments
 (0)