Skip to content

Commit a593b52

Browse files
authored
Merge pull request #1696 from ClickHouse/add-nested-serializer
Updating Nested type code and fixing proxy test
2 parents 1ce84b4 + 160609e commit a593b52

File tree

8 files changed

+102
-137
lines changed

8 files changed

+102
-137
lines changed

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

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -589,7 +589,7 @@ private ClickHouseColumn(ClickHouseDataType dataType, String columnName, String
589589
this.originalTypeName = originalTypeName == null ? dataType.name() : originalTypeName;
590590
this.nullable = nullable;
591591
this.lowCardinality = lowCardinality;
592-
this.hasDefault = originalTypeName != null && originalTypeName.toUpperCase().contains("DEFAULT");
592+
this.hasDefault = false;
593593

594594
if (parameters == null || parameters.isEmpty()) {
595595
this.parameters = Collections.emptyList();
@@ -734,6 +734,10 @@ public boolean hasDefault() {
734734
return hasDefault;
735735
}
736736

737+
public void setHasDefault(boolean hasDefault) {
738+
this.hasDefault = hasDefault;
739+
}
740+
737741
public boolean isLowCardinality() {
738742
return !lowCardinalityDisabled && lowCardinality;
739743
}

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

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,7 @@
11
package com.clickhouse.client.api;
22

33
import com.clickhouse.client.ClickHouseClient;
4-
import com.clickhouse.client.ClickHouseClientBuilder;
5-
import com.clickhouse.client.ClickHouseConfig;
64
import com.clickhouse.client.ClickHouseNode;
7-
import com.clickhouse.client.ClickHouseNodeSelector;
8-
import com.clickhouse.client.ClickHouseProtocol;
95
import com.clickhouse.client.ClickHouseRequest;
106
import com.clickhouse.client.ClickHouseResponse;
117
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
@@ -31,7 +27,6 @@
3127
import com.clickhouse.client.api.query.QuerySettings;
3228
import com.clickhouse.client.api.query.Records;
3329
import com.clickhouse.client.config.ClickHouseClientOption;
34-
import com.clickhouse.config.ClickHouseOption;
3530
import com.clickhouse.data.ClickHouseColumn;
3631
import com.clickhouse.data.ClickHouseDataStreamFactory;
3732
import com.clickhouse.data.ClickHouseFormat;
@@ -44,7 +39,6 @@
4439
import java.io.ByteArrayOutputStream;
4540
import java.io.IOException;
4641
import java.io.InputStream;
47-
import java.io.Serializable;
4842
import java.lang.reflect.InvocationTargetException;
4943
import java.lang.reflect.Method;
5044
import java.net.URL;
@@ -465,7 +459,7 @@ public void register(Class<?> clazz, TableSchema schema) {
465459
this.getterMethods.put(clazz, getterMethods);//Store the getter methods for later use
466460

467461
for (ClickHouseColumn column : schema.getColumns()) {
468-
String columnName = column.getColumnName().toLowerCase().replace("_", "");
462+
String columnName = column.getColumnName().toLowerCase().replace("_", "").replace(".","");
469463
serializers.add((obj, stream) -> {
470464
if (!getterMethods.containsKey(columnName)) {
471465
LOG.warn("No getter method found for column: {}", columnName);

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

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -36,9 +36,6 @@ public static void serializeData(OutputStream stream, Object value, ClickHouseCo
3636
case Map:
3737
serializeMapData(stream, value, column);
3838
break;
39-
case Nested:
40-
LOG.error("Nested data type is not supported.");
41-
break;
4239
default:
4340
serializePrimitiveData(stream, value, column);
4441
break;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ public TableSchema createFromBinaryResponse(ClickHouseResponse response, String
2222
try {
2323
p.clear();
2424
p.load(new StringReader(values));
25-
schema.addColumn(p.getProperty("name"), p.getProperty("type"));
25+
schema.addColumn(p.getProperty("name"), p.getProperty("type"), p.getProperty("default_type"));
2626
} catch ( IOException e) {
2727
throw new RuntimeException("Failed to parse table schema", e);
2828
}

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

Lines changed: 20 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -58,10 +58,16 @@ public boolean hasDefaults() {
5858
}
5959

6060
public void addColumn(String name, String type) {
61-
columns.add(ClickHouseColumn.of(name, type));
62-
if (type.toUpperCase().contains("DEFAULT")) {
61+
addColumn(name, type, "");
62+
}
63+
public void addColumn(String name, String type, String defaultType) {
64+
ClickHouseColumn column = ClickHouseColumn.of(name, type);
65+
if (defaultType.toUpperCase().contains("DEFAULT")) {
6366
hasDefaults = true;
67+
column.setHasDefault(true);
6468
}
69+
columns.add(column);
70+
6571
Map<String, Object> columnMetadata = metadata.computeIfAbsent(name, k -> new HashMap<>());
6672
columnMetadata.put("type", type);
6773
colIndex.put(name, columns.size() - 1);
@@ -84,5 +90,17 @@ public String indexToName(int index) {
8490
public int nameToIndex(String name) {
8591
return colIndex.get(name).intValue();
8692
}
93+
94+
@Override
95+
public String toString() {
96+
return "TableSchema{" +
97+
"tableName='" + tableName + '\'' +
98+
", databaseName='" + databaseName + '\'' +
99+
", columns=" + columns +
100+
", metadata=" + metadata +
101+
", colIndex=" + colIndex +
102+
", hasDefaults=" + hasDefaults +
103+
'}';
104+
}
87105
}
88106

client-v2/src/test/java/com/clickhouse/client/ProxyTests.java

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import static org.testng.Assert.assertEquals;
2626
import static org.testng.Assert.assertThrows;
2727
import static org.testng.Assert.assertTrue;
28+
import static org.testng.Assert.fail;
2829

2930
public class ProxyTests extends BaseIntegrationTest{
3031
private Client client;
@@ -69,14 +70,14 @@ private void createTable(String tableQuery) throws ClickHouseException {
6970
}
7071

7172

72-
@Test(groups = { "integration" }, enabled = false)
73+
@Test(groups = { "integration" }, enabled = true)
7374
public void simpleProxyTest() throws Exception {
74-
String tableName = "simple_pojo_proxy_table";
75+
String tableName = "simple_pojo_enable_proxy_table";
7576
String createSQL = SamplePOJO.generateTableCreateSQL(tableName);
7677
System.out.println(createSQL);
7778
createTable(createSQL);
7879

79-
client.register(SamplePOJO.class, SamplePOJO.generateTableSchema(tableName));
80+
client.register(SamplePOJO.class, client.getTableSchema(tableName, "default"));
8081
List<Object> simplePOJOs = new ArrayList<>();
8182

8283
for (int i = 0; i < 1000; i++) {
@@ -94,24 +95,23 @@ public void simpleProxyTest() throws Exception {
9495

9596
@Test(groups = { "integration" }, enabled = true)
9697
public void simpleDisabledProxyTest() throws Exception {
97-
String tableName = "simple_pojo_proxy_table";
98+
String tableName = "simple_pojo_disable_proxy_table";
9899
String createSQL = SamplePOJO.generateTableCreateSQL(tableName);
99100
System.out.println(createSQL);
100101
createTable(createSQL);
101102

102-
client.register(SamplePOJO.class, SamplePOJO.generateTableSchema(tableName));
103+
client.register(SamplePOJO.class, client.getTableSchema(tableName, "default"));
103104
List<Object> simplePOJOs = new ArrayList<>();
104105

105106
for (int i = 0; i < 1000; i++) {
106107
simplePOJOs.add(new SamplePOJO());
107108
}
108-
//proxy.disable();
109+
proxy.disable();
109110
try {
110111
InsertResponse response = client.insert(tableName, simplePOJOs).get(120, TimeUnit.SECONDS);
112+
fail("Should have thrown exception.");
111113
} catch (Exception e) {
112-
//FOR NOW THIS WILL FAIL
113-
//IF IT SUCCEEDS (meaning the test fails), YAY PROXY WORKS WE SHOULD REIMPLEMENT THE TESTS!
114-
assertTrue(e.getMessage().contains("Operation has likely timed out."));
114+
assertTrue(e instanceof ClientException);
115115
}
116116
}
117117
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111
import com.clickhouse.client.api.enums.Protocol;
1212
import com.clickhouse.client.api.insert.InsertResponse;
1313
import com.clickhouse.client.api.insert.InsertSettings;
14+
import com.clickhouse.client.api.metadata.TableSchema;
1415
import com.clickhouse.client.api.metrics.ClientMetrics;
1516
import com.clickhouse.client.api.metrics.OperationMetrics;
1617
import com.clickhouse.client.api.metrics.ServerMetrics;
@@ -58,8 +59,7 @@ public void insertSimplePOJOs() throws Exception {
5859
String createSQL = SamplePOJO.generateTableCreateSQL(tableName);
5960
System.out.println(createSQL);
6061
createTable(createSQL);
61-
62-
client.register(SamplePOJO.class, SamplePOJO.generateTableSchema(tableName));
62+
client.register(SamplePOJO.class, client.getTableSchema(tableName, "default"));
6363
List<Object> simplePOJOs = new ArrayList<>();
6464

6565
for (int i = 0; i < 1000; i++) {

0 commit comments

Comments
 (0)