Skip to content

Commit f7ac19c

Browse files
author
Paultagoras
committed
Adding support for defaults to serialization
1 parent ea8c680 commit f7ac19c

File tree

4 files changed

+79
-6
lines changed

4 files changed

+79
-6
lines changed

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

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -74,6 +74,7 @@ public final class ClickHouseColumn implements Serializable {
7474
private ClickHouseAggregateFunction aggFuncType;
7575
private ClickHouseDataType dataType;
7676
private boolean nullable;
77+
private boolean hasDefault;
7778
private boolean lowCardinality;
7879
private boolean lowCardinalityDisabled;
7980
private TimeZone timeZone;
@@ -588,6 +589,7 @@ private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String
588589
this.originalTypeName = originalTypeName == null ? dataType.name() : originalTypeName;
589590
this.nullable = nullable;
590591
this.lowCardinality = lowCardinality;
592+
this.hasDefault = originalTypeName != null && originalTypeName.toUpperCase().contains("DEFAULT");
591593

592594
if (parameters == null || parameters.isEmpty()) {
593595
this.parameters = Collections.emptyList();
@@ -728,6 +730,10 @@ public boolean isNullable() {
728730
return nullable;
729731
}
730732

733+
public boolean hasDefault() {
734+
return hasDefault;
735+
}
736+
731737
public boolean isLowCardinality() {
732738
return !lowCardinalityDisabled && lowCardinality;
733739
}

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

Lines changed: 25 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@ public class Client {
4848
private List<ClickHouseNode> serverNodes = new ArrayList<>();
4949
private Map<Class<?>, List<POJOSerializer>> serializers;//Order is important to preserve for RowBinary
5050
private Map<Class<?>, Map<String, Method>> getterMethods;
51+
private Map<Class<?>, Boolean> hasDefaults;
5152
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
5253

5354
private Client(Set<String> endpoints, Map<String,String> configuration) {
@@ -58,6 +59,7 @@ private Client(Set<String> endpoints, Map<String,String> configuration) {
5859
});
5960
this.serializers = new HashMap<>();
6061
this.getterMethods = new HashMap<>();
62+
this.hasDefaults = new HashMap<>();
6163
}
6264

6365
public static class Builder {
@@ -206,22 +208,33 @@ public void register(Class<?> clazz, TableSchema schema) {
206208
}
207209
Method getterMethod = this.getterMethods.get(clazz).get(columnName);
208210
Object value = getterMethod.invoke(obj);
211+
boolean hasDefaults = this.hasDefaults.get(clazz);
209212

210213
//Handle null values
211214
if (value == null) {
212-
BinaryStreamUtils.writeNull(stream);
213-
return;
214-
} else {//If nullable, we have to specify that the value is not null
215-
if (column.isNullable()) {
215+
if (hasDefaults && !column.hasDefault()) {//Send this only if there is no default
216216
BinaryStreamUtils.writeNonNull(stream);
217217
}
218+
BinaryStreamUtils.writeNull(stream);//We send this regardless of default or nullable
219+
return;
220+
}
221+
222+
//Handle default
223+
if (hasDefaults) {
224+
BinaryStreamUtils.writeNonNull(stream);//Write 0
225+
}
226+
227+
//Handle nullable
228+
if (column.isNullable()) {
229+
BinaryStreamUtils.writeNonNull(stream);//Write 0
218230
}
219231

220232
//Handle the different types
221233
SerializerUtils.serializeData(stream, value, column);
222234
});
223235
}
224236
this.serializers.put(clazz, serializers);
237+
this.hasDefaults.put(clazz, schema.hasDefaults());
225238
}
226239

227240
/**
@@ -239,7 +252,14 @@ public InsertResponse insert(String tableName,
239252
if (settings == null) {
240253
settings = new InsertSettings();
241254
}
242-
settings.setFormat(ClickHouseFormat.RowBinary);
255+
256+
boolean hasDefaults = this.hasDefaults.get(data.get(0).getClass());
257+
if (hasDefaults) {
258+
settings.setFormat(ClickHouseFormat.RowBinaryWithDefaults);
259+
} else {
260+
settings.setFormat(ClickHouseFormat.RowBinary);
261+
}
262+
243263

244264
//Create an output stream to write the data to
245265
ByteArrayOutputStream stream = new ByteArrayOutputStream();

client-v2/src/main/java/com/clickhouse/client/api/metadata/TableSchema.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@ public class TableSchema {
1818

1919
private Map<String, Map<String, Object>> metadata;
2020

21+
private boolean hasDefaults = false;
22+
2123
public TableSchema() {
2224
this.metadata = new HashMap<>();
2325
this.columns = new ArrayList<>();
@@ -48,8 +50,15 @@ public void setDatabaseName(String databaseName) {
4850
this.databaseName = databaseName;
4951
}
5052

53+
public boolean hasDefaults() {
54+
return hasDefaults;
55+
}
56+
5157
public void addColumn(String name, String type) {
5258
columns.add(ClickHouseColumn.of(name, type));
59+
if (type.toUpperCase().contains("DEFAULT")) {
60+
hasDefaults = true;
61+
}
5362
metadata.computeIfAbsent(name, k -> new HashMap<>()).put("type", type);
5463
}
5564

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

Lines changed: 39 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,7 @@
11
package com.clickhouse.client.insert;
22

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.data.ClickHouseColumn;
45
import com.clickhouse.data.ClickHouseEnum;
56
import org.apache.commons.lang3.RandomStringUtils;
67

@@ -19,11 +20,17 @@
1920

2021
public class SamplePOJO {
2122
private int int8;
23+
private int int8_default;
2224
private int int16;
25+
private int int16_default;
2326
private int int32;
27+
private int int32_default;
2428
private long int64;
29+
private long int64_default;
2530
private BigInteger int128;
31+
private BigInteger int128_default;
2632
private BigInteger int256;
33+
private BigInteger int256_default;
2734

2835
private int uint8;
2936
private int uint16;
@@ -132,6 +139,10 @@ public void setInt8(int int8) {
132139
this.int8 = int8;
133140
}
134141

142+
public int getInt8Default() {
143+
return int8_default;
144+
}
145+
135146
public int getInt16() {
136147
return int16;
137148
}
@@ -140,10 +151,18 @@ public void setInt16(int int16) {
140151
this.int16 = int16;
141152
}
142153

154+
public int getInt16Default() {
155+
return int16_default;
156+
}
157+
143158
public int getInt32() {
144159
return int32;
145160
}
146161

162+
public int getInt32Default() {
163+
return int32_default;
164+
}
165+
147166
public void setInt32(int int32) {
148167
this.int32 = int32;
149168
}
@@ -152,6 +171,10 @@ public long getInt64() {
152171
return int64;
153172
}
154173

174+
public long getInt64Default() {
175+
return int64_default;
176+
}
177+
155178
public void setInt64(long int64) {
156179
this.int64 = int64;
157180
}
@@ -160,6 +183,10 @@ public BigInteger getInt128() {
160183
return int128;
161184
}
162185

186+
public BigInteger getInt128Default() {
187+
return int128_default;
188+
}
189+
163190
public void setInt128(BigInteger int128) {
164191
this.int128 = int128;
165192
}
@@ -168,6 +195,10 @@ public BigInteger getInt256() {
168195
return int256;
169196
}
170197

198+
public BigInteger getInt256Default() {
199+
return int256_default;
200+
}
201+
171202
public void setInt256(BigInteger int256) {
172203
this.int256 = int256;
173204
}
@@ -401,11 +432,17 @@ public static TableSchema generateTableSchema(String tableName) {
401432
schema.setDatabaseName("default");
402433
schema.setTableName(tableName);
403434
schema.addColumn("int8", "Int8");
435+
schema.addColumn("int8_default", "Int8 DEFAULT 0");
404436
schema.addColumn("int16", "Int16");
437+
schema.addColumn("int16_default", "Int16 DEFAULT 0");
405438
schema.addColumn("int32", "Int32");
439+
schema.addColumn("int32_default", "Int32 DEFAULT 0");
406440
schema.addColumn("int64", "Int64");
441+
schema.addColumn("int64_default", "Int64 DEFAULT 0");
407442
schema.addColumn("int128", "Int128");
443+
schema.addColumn("int128_default", "Int128 DEFAULT 0");
408444
schema.addColumn("int256", "Int256");
445+
schema.addColumn("int256_default", "Int256 DEFAULT 0");
409446

410447
schema.addColumn("uint8", "UInt8");
411448
schema.addColumn("uint16", "UInt16");
@@ -458,7 +495,8 @@ public static String generateTableCreateSQL(String tableName) {
458495
if (i > 0) {
459496
sb.append(", ");
460497
}
461-
sb.append(schema.getColumns().get(i).getColumnName()).append(" ").append(schema.getColumns().get(i).getOriginalTypeName());
498+
ClickHouseColumn column = schema.getColumns().get(i);
499+
sb.append(column.getColumnName()).append(" ").append(column.getOriginalTypeName()).append(column.hasDefault() ? 0 : "");
462500
}
463501
sb.append(") ENGINE = Memory");
464502
return sb.toString();

0 commit comments

Comments
 (0)