Skip to content

Commit e00a973

Browse files
committed
added clientV1 RowBinary serialization
1 parent 645f3a5 commit e00a973

File tree

7 files changed

+145
-16
lines changed

7 files changed

+145
-16
lines changed

performance/src/test/com/clickhouse/benchmark/BenchmarkRunner.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ public static void main(String[] args) throws Exception {
4747
.include(InsertClient.class.getSimpleName())
4848
// .include(JdbcV1.class.getSimpleName())
4949
// .include(JdbcV2.class.getSimpleName())
50-
.forks(1) // must be a fork. No fork only for debugging
50+
.forks(0) // must be a fork. No fork only for debugging
5151
.mode(Mode.AverageTime)
5252
.timeUnit(TimeUnit.MILLISECONDS)
5353
.threads(1)

performance/src/test/com/clickhouse/benchmark/clients/BenchmarkBase.java

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -5,20 +5,30 @@
55
import com.clickhouse.benchmark.data.FileDataSet;
66
import com.clickhouse.benchmark.data.SimpleDataSet;
77
import com.clickhouse.client.BaseIntegrationTest;
8+
import com.clickhouse.client.ClickHouseClient;
9+
import com.clickhouse.client.ClickHouseCredentials;
810
import com.clickhouse.client.ClickHouseNode;
911
import com.clickhouse.client.ClickHouseProtocol;
12+
import com.clickhouse.client.ClickHouseResponse;
1013
import com.clickhouse.client.ClickHouseServerForTest;
1114
import com.clickhouse.client.api.Client;
1215
import com.clickhouse.client.api.enums.Protocol;
1316
import com.clickhouse.client.api.insert.InsertResponse;
17+
import com.clickhouse.data.ClickHouseDataProcessor;
1418
import com.clickhouse.data.ClickHouseFormat;
19+
import com.clickhouse.data.ClickHouseOutputStream;
20+
import com.clickhouse.data.ClickHouseRecord;
21+
import com.clickhouse.data.format.ClickHouseRowBinaryProcessor;
1522
import org.openjdk.jmh.annotations.Param;
1623
import org.openjdk.jmh.annotations.Scope;
1724
import org.openjdk.jmh.annotations.State;
1825
import org.slf4j.Logger;
1926
import org.slf4j.LoggerFactory;
2027

28+
import java.io.ByteArrayOutputStream;
2129
import java.io.InputStream;
30+
import java.util.ArrayList;
31+
import java.util.Collections;
2232

2333
import static com.clickhouse.client.ClickHouseServerForTest.isCloud;
2434

@@ -41,7 +51,6 @@ public void setup(DataState dataState, boolean insertData) throws Exception {
4151
dataState.datasetSourceName = "simple";
4252
dataState.dataSet = new SimpleDataSet();
4353
} else if (dataState.datasetSourceName.startsWith("file://")) {
44-
4554
dataState.dataSet = new FileDataSet(dataState.datasetSourceName.substring("file://".length()));
4655
dataState.datasetSourceName = dataState.dataSet.getName();
4756
}
@@ -102,4 +111,31 @@ public static void insertData(String tableName, InputStream dataStream, ClickHou
102111
throw new RuntimeException("Error inserting data", e);
103112
}
104113
}
114+
115+
public static void loadClickHouseRecords(String tableName, DataSet dataSet) {
116+
ClickHouseNode node = getServer();
117+
118+
try (ClickHouseClient clientV1 = ClickHouseClient
119+
.newInstance(ClickHouseCredentials.fromUserAndPassword(getUsername(), getPassword()), ClickHouseProtocol.HTTP);
120+
ClickHouseResponse response = clientV1.read(node).query("SELECT * FROM " + DB_NAME + "." + tableName)
121+
.format(ClickHouseFormat.RowBinaryWithNamesAndTypes)
122+
.executeAndWait()) {
123+
124+
// Create a data processor to serialize data in ClientV1 tests
125+
ClickHouseDataProcessor dataProcessor= new ClickHouseRowBinaryProcessor(clientV1.getConfig(), null,
126+
ClickHouseOutputStream.of(new ByteArrayOutputStream()), response.getColumns(), Collections.emptyMap());
127+
assert dataProcessor.getColumns() != null;
128+
dataSet.setClickHouseDataProcessor(dataProcessor);
129+
130+
ArrayList<ClickHouseRecord> records = new ArrayList<>();
131+
for (ClickHouseRecord record : response.records()) {
132+
records.add(record);
133+
}
134+
135+
dataSet.setClickHouseRecords(records);
136+
} catch (Exception e) {
137+
LOGGER.error("Error inserting data: ", e);
138+
throw new RuntimeException("Error inserting data", e);
139+
}
140+
}
105141
}

performance/src/test/com/clickhouse/benchmark/clients/InsertClient.java

Lines changed: 35 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -11,14 +11,14 @@
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.query.GenericRecord;
1514
import com.clickhouse.client.api.query.QueryResponse;
1615
import com.clickhouse.client.config.ClickHouseClientOption;
16+
import com.clickhouse.data.ClickHouseDataProcessor;
1717
import com.clickhouse.data.ClickHouseFormat;
1818
import com.clickhouse.data.ClickHouseRecord;
19+
import com.clickhouse.data.ClickHouseSerializer;
1920
import org.openjdk.jmh.annotations.Benchmark;
2021
import org.openjdk.jmh.annotations.Level;
21-
import org.openjdk.jmh.annotations.Param;
2222
import org.openjdk.jmh.annotations.Scope;
2323
import org.openjdk.jmh.annotations.Setup;
2424
import org.openjdk.jmh.annotations.State;
@@ -92,15 +92,9 @@ public void tearDownIteration(DataState dataState) throws InterruptedException {
9292
LOGGER.error("Error: ", e);
9393
}
9494
}
95-
@State(Scope.Thread)
96-
public static class InsertState {
97-
@Param({"simple"})
98-
String dataSetName;
99-
ClickHouseFormat format = ClickHouseFormat.JSONEachRow;
100-
}
10195

10296
@Benchmark
103-
public void insertV1(DataState dataState, InsertState state) {
97+
public void insertV1(DataState dataState) {
10498
try {
10599
ClickHouseFormat format = dataState.dataSet.getFormat();
106100
try (ClickHouseResponse response = clientV1.read(getServer())
@@ -125,7 +119,7 @@ public void insertV1(DataState dataState, InsertState state) {
125119
}
126120

127121
@Benchmark
128-
public void insertV2(DataState dataState, InsertState state) {
122+
public void insertV2(DataState dataState) {
129123
try {
130124
ClickHouseFormat format = dataState.dataSet.getFormat();
131125
try (InsertResponse response = clientV2.insert(dataState.dataSet.getTableName(), out -> {
@@ -143,4 +137,35 @@ public void insertV2(DataState dataState, InsertState state) {
143137
LOGGER.error("Error: ", e);
144138
}
145139
}
140+
141+
@Benchmark
142+
public void insertV1RowBinary(DataState dataState) {
143+
try {
144+
ClickHouseFormat format = ClickHouseFormat.RowBinary;
145+
try (ClickHouseResponse response = clientV1.read(getServer())
146+
.write()
147+
.option(ClickHouseClientOption.ASYNC, false)
148+
.format(format)
149+
.query("INSERT INTO `" + DB_NAME + "`.`" + dataState.dataSet.getTableName() + "`")
150+
.data(out -> {
151+
ClickHouseDataProcessor p = dataState.dataSet.getClickHouseDataProcessor();
152+
ClickHouseSerializer[] serializers = p.getSerializers(clientV1.getConfig(), p.getColumns());
153+
154+
for (ClickHouseRecord record : dataState.dataSet.getClickHouseRecords()) {
155+
for (int i = 0; i < serializers.length; i++) {
156+
serializers[i].serialize(record.getValue(i), out);
157+
}
158+
}
159+
160+
})
161+
.executeAndWait()) {
162+
ClickHouseResponseSummary summary = response.getSummary();
163+
if (summary.getWrittenRows() <= 0) {
164+
throw new RuntimeException("Rows written: " + summary.getWrittenRows());
165+
}
166+
}
167+
} catch ( Exception e) {
168+
LOGGER.error("Error: ", e);
169+
}
170+
}
146171
}

performance/src/test/com/clickhouse/benchmark/data/DataSet.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,9 @@
11
package com.clickhouse.benchmark.data;
22

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.data.ClickHouseDataProcessor;
45
import com.clickhouse.data.ClickHouseFormat;
6+
import com.clickhouse.data.ClickHouseRecord;
57

68
import java.io.ByteArrayInputStream;
79
import java.io.ByteArrayOutputStream;
@@ -41,4 +43,12 @@ default InputStream getInputStream(int rowId, ClickHouseFormat format) {
4143
List<byte[]> getBytesList(ClickHouseFormat format);
4244

4345
List<Map<String, Object>> getRows();
46+
47+
List<ClickHouseRecord> getClickHouseRecords();
48+
49+
void setClickHouseRecords(List<ClickHouseRecord> records);
50+
51+
void setClickHouseDataProcessor(ClickHouseDataProcessor dataProcessor);
52+
53+
ClickHouseDataProcessor getClickHouseDataProcessor();
4454
}

performance/src/test/com/clickhouse/benchmark/data/DataSets.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,11 @@ public static DataSet from(String name) {
2727
public static void initializeTables(DataSet set, boolean insertData) {
2828
BenchmarkBase.runQuery(set.getCreateTableString(), true);
2929
ClickHouseFormat format = set.getFormat();
30-
if (insertData) {
31-
BenchmarkBase.insertData(set.getTableName(), set.getInputStream(format), format);
30+
31+
BenchmarkBase.insertData(set.getTableName(), set.getInputStream(format), format);
32+
if (!insertData) {
33+
BenchmarkBase.loadClickHouseRecords(set.getTableName(), set);
34+
BenchmarkBase.runQuery("TRUNCATE TABLE " + set.getTableName(), true);
3235
}
3336
}
3437

performance/src/test/com/clickhouse/benchmark/data/FileDataSet.java

Lines changed: 26 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,13 +1,14 @@
11
package com.clickhouse.benchmark.data;
22

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.data.ClickHouseDataProcessor;
45
import com.clickhouse.data.ClickHouseFormat;
6+
import com.clickhouse.data.ClickHouseRecord;
57
import org.slf4j.Logger;
68
import org.slf4j.LoggerFactory;
79

810
import java.io.BufferedReader;
911
import java.io.File;
10-
import java.io.InputStream;
1112
import java.util.ArrayList;
1213
import java.util.Collections;
1314
import java.util.HashMap;
@@ -122,6 +123,30 @@ public ClickHouseFormat getFormat() {
122123
return ClickHouseFormat.CSV;
123124
}
124125

126+
private List<ClickHouseRecord> clickHouseRecords;
127+
128+
@Override
129+
public List<ClickHouseRecord> getClickHouseRecords() {
130+
return clickHouseRecords;
131+
}
132+
133+
@Override
134+
public void setClickHouseRecords(List<ClickHouseRecord> records) {
135+
this.clickHouseRecords = records;
136+
}
137+
138+
private ClickHouseDataProcessor dataProcessor;
139+
140+
@Override
141+
public ClickHouseDataProcessor getClickHouseDataProcessor() {
142+
return dataProcessor;
143+
}
144+
145+
@Override
146+
public void setClickHouseDataProcessor(ClickHouseDataProcessor dataProcessor) {
147+
this.dataProcessor = dataProcessor;
148+
}
149+
125150
@Override
126151
public String toString() {
127152
return "FileDataSet{" +

performance/src/test/com/clickhouse/benchmark/data/SimpleDataSet.java

Lines changed: 31 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,9 @@
11
package com.clickhouse.benchmark.data;
22

33
import com.clickhouse.client.api.metadata.TableSchema;
4+
import com.clickhouse.data.ClickHouseDataProcessor;
45
import com.clickhouse.data.ClickHouseFormat;
6+
import com.clickhouse.data.ClickHouseRecord;
57
import org.slf4j.Logger;
68
import org.slf4j.LoggerFactory;
79

@@ -26,7 +28,7 @@ public class SimpleDataSet implements DataSet {
2628

2729
public SimpleDataSet() {
2830
tableName = name + "_dataset_" + UUID.randomUUID().toString().replaceAll("-", "");
29-
size = 100000;
31+
size = 10;
3032

3133
data = new ArrayList<>(size);
3234
for (int i = 0; i < size; i++) {
@@ -95,10 +97,12 @@ public String getCreateTableString() {
9597
"ENGINE = MergeTree\n" +
9698
"PRIMARY KEY (pickup_datetime, dropoff_datetime);";
9799
}
100+
98101
@Override
99102
public String getTrucateTableString() {
100103
return "TRUNCATE TABLE " + getTableName();
101104
}
105+
102106
@Override
103107
public ClickHouseFormat getFormat() {
104108
return ClickHouseFormat.JSONEachRow;
@@ -149,4 +153,30 @@ public List<byte[]> getBytesList(ClickHouseFormat format) {
149153
public List<Map<String, Object>> getRows() {
150154
return data;
151155
}
156+
157+
158+
private List<ClickHouseRecord> clickHouseRecords;
159+
160+
@Override
161+
public List<ClickHouseRecord> getClickHouseRecords() {
162+
return clickHouseRecords;
163+
}
164+
165+
@Override
166+
public void setClickHouseRecords(List<ClickHouseRecord> records) {
167+
clickHouseRecords = records;
168+
}
169+
170+
171+
private ClickHouseDataProcessor dataProcessor;
172+
173+
@Override
174+
public ClickHouseDataProcessor getClickHouseDataProcessor() {
175+
return dataProcessor;
176+
}
177+
178+
@Override
179+
public void setClickHouseDataProcessor(ClickHouseDataProcessor dataProcessor) {
180+
this.dataProcessor = dataProcessor;
181+
}
152182
}

0 commit comments

Comments
 (0)