Skip to content

Commit 17681b3

Browse files
authored
Merge pull request #2243 from ClickHouse/perf_typed_tests
[perf] Added tests for data types (DateTime)
2 parents 5945902 + cdee17f commit 17681b3

File tree

5 files changed

+266
-56
lines changed

5 files changed

+266
-56
lines changed

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

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

33
import com.clickhouse.benchmark.clients.Compression;
4+
import com.clickhouse.benchmark.clients.DataTypes;
45
import com.clickhouse.benchmark.clients.ConcurrentInsertClient;
56
import com.clickhouse.benchmark.clients.ConcurrentQueryClient;
67
import com.clickhouse.benchmark.clients.Deserializers;
@@ -46,6 +47,7 @@ public static void main(String[] args) throws Exception {
4647
.include(Serializers.class.getName())
4748
.include(Deserializers.class.getName())
4849
.include(MixedWorkload.class.getName())
50+
.include(DataTypes.class.getName())
4951
.include(JDBCQuery.class.getName())
5052
.include(JDBCInsert.class.getName())
5153
.forks(1) // must be a fork. No fork only for debugging

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

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,7 @@
44
import com.clickhouse.benchmark.data.DataSet;
55
import com.clickhouse.benchmark.data.FileDataSet;
66
import com.clickhouse.benchmark.data.SimpleDataSet;
7+
import com.clickhouse.benchmark.data.SyntheticDataSet;
78
import com.clickhouse.client.ClickHouseClient;
89
import com.clickhouse.client.ClickHouseClientBuilder;
910
import com.clickhouse.client.ClickHouseCredentials;
@@ -117,6 +118,8 @@ public static class DataState {
117118

118119
ByteBuffer datasetAsRowBinaryWithNamesAndTypes;
119120

121+
SyntheticDataSet syntheticDataSet;
122+
120123
public void setDataSet(DataSet dataSet) {
121124
this.dataSet = dataSet;
122125
}
Lines changed: 171 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,171 @@
1+
package com.clickhouse.benchmark.clients;
2+
3+
import com.clickhouse.benchmark.data.SyntheticDataSet;
4+
import com.clickhouse.client.api.Client;
5+
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
6+
import com.clickhouse.client.api.data_formats.internal.SerializerUtils;
7+
import com.clickhouse.client.api.query.QueryResponse;
8+
import com.clickhouse.client.api.query.QuerySettings;
9+
import com.clickhouse.data.ClickHouseColumn;
10+
import com.clickhouse.data.ClickHouseFormat;
11+
import com.clickhouse.data.ClickHouseInputStream;
12+
import com.clickhouse.data.ClickHouseOutputStream;
13+
import com.clickhouse.data.format.BinaryDataProcessor;
14+
import com.clickhouse.data.format.BinaryStreamUtils;
15+
import com.clickhouse.data.value.ClickHouseDateTimeValue;
16+
import org.openjdk.jmh.annotations.Benchmark;
17+
import org.openjdk.jmh.annotations.Level;
18+
import org.openjdk.jmh.annotations.Setup;
19+
import org.openjdk.jmh.infra.Blackhole;
20+
import org.slf4j.Logger;
21+
import org.slf4j.LoggerFactory;
22+
23+
import java.io.EOFException;
24+
import java.io.IOException;
25+
import java.io.OutputStream;
26+
import java.nio.ByteBuffer;
27+
import java.time.LocalDateTime;
28+
import java.util.TimeZone;
29+
30+
public class DataTypes extends BenchmarkBase {
31+
32+
private static final Logger LOGGER = LoggerFactory.getLogger(DataTypes.class);
33+
34+
@Setup(Level.Iteration)
35+
public void setUpIteration(DataState dataState) {
36+
super.setUpIteration();
37+
38+
try (Client c = getClientV2(); QueryResponse r = c.query("SELECT * FROM " + dataState.tableNameFilled, new QuerySettings()
39+
.setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes)).get()) {
40+
dataState.datasetAsRowBinaryWithNamesAndTypes = ByteBuffer.wrap(r.getInputStream().readAllBytes());
41+
LOGGER.info("Loaded {} from dataset", dataState.datasetAsRowBinaryWithNamesAndTypes.capacity());
42+
} catch (Exception e) {
43+
LOGGER.error("Failed to init data for components benchmark", e);
44+
}
45+
46+
if (dataState.syntheticDataSet != null) {
47+
dataState.syntheticDataSet = new SyntheticDataSet(dataState.limit);
48+
}
49+
}
50+
51+
@Benchmark
52+
public void readDateTimeV1(DataState dataState, Blackhole blackhole) {
53+
ClickHouseInputStream input = ClickHouseInputStream.of(dataState.syntheticDataSet.getDateTimeValuesRowBinaryStream());
54+
BinaryDataProcessor.DateTime64SerDe serDe = new BinaryDataProcessor.DateTime64SerDe(3, TimeZone.getTimeZone("UTC"));
55+
56+
ClickHouseDateTimeValue valueHolder = ClickHouseDateTimeValue.ofNull(3, TimeZone.getTimeZone("UTC"));
57+
58+
int valueCount = 0;
59+
while (valueCount <= dataState.limit) {
60+
try {
61+
serDe.deserialize(valueHolder, input);
62+
blackhole.consume(valueHolder);
63+
valueCount++;
64+
} catch (IOException ex) {
65+
if (valueCount < dataState.limit) {
66+
throw new RuntimeException("Failed to read all values", ex);
67+
}
68+
break;
69+
}
70+
}
71+
}
72+
73+
@Benchmark
74+
public void readDateTimeV2(DataState dataState, Blackhole blackhole) {
75+
ClickHouseInputStream input = ClickHouseInputStream.of(dataState.syntheticDataSet.getDateTimeValuesRowBinaryStream());
76+
77+
byte[] buffer = new byte[8];
78+
TimeZone zoneId = TimeZone.getTimeZone("UTC");
79+
80+
int valueCount = 0;
81+
while (valueCount <= dataState.limit) {
82+
try {
83+
blackhole.consume(BinaryStreamReader.readDateTime64(input, buffer, 3, zoneId));
84+
valueCount++;
85+
} catch (EOFException ex) {
86+
if (valueCount < dataState.limit) {
87+
throw new RuntimeException("Failed to read all values", ex);
88+
}
89+
break;
90+
} catch (IOException ex) {
91+
throw new RuntimeException("Failed to read all values", ex);
92+
}
93+
}
94+
}
95+
96+
97+
@Benchmark
98+
public void DateTimeSerializerV1(DataState dataState, Blackhole blackhole) {
99+
OutputStream empty = new BlackholeOutputStream(blackhole);
100+
BinaryDataProcessor.DateTime64SerDe serDe =
101+
new BinaryDataProcessor.DateTime64SerDe(3, TimeZone.getTimeZone("UTC"));
102+
103+
ClickHouseOutputStream chos = ClickHouseOutputStream.of(empty);
104+
TimeZone tz = TimeZone.getTimeZone("UTC");
105+
106+
for (LocalDateTime dateTime : dataState.syntheticDataSet.getDateTimeValues()) {
107+
try {
108+
BinaryStreamUtils.writeDateTime64(chos, dateTime, 3, tz);
109+
// App should wrap a value with a value object if it wants to use a data processor
110+
// serDe.serialize(ClickHouseDateTimeValue.of(dateTime, 3, tz) , chos);
111+
} catch (Exception e) {
112+
LOGGER.error("Error: ", e);
113+
}
114+
}
115+
try {
116+
chos.flush();
117+
} catch (Exception e) {
118+
LOGGER.error("Error: ", e);
119+
}
120+
}
121+
122+
@Benchmark
123+
public void DateTimeSerializerV2(DataState dataState, Blackhole blackhole) {
124+
OutputStream empty = new BlackholeOutputStream(blackhole);
125+
ClickHouseColumn column = ClickHouseColumn.of("a", "DateTime64(3, 'UTC')");
126+
127+
for (LocalDateTime dateTime : dataState.syntheticDataSet.getDateTimeValues()) {
128+
try {
129+
SerializerUtils.serializeData(empty, dateTime, column);
130+
} catch (Exception e) {
131+
LOGGER.error("Error: ", e);
132+
}
133+
}
134+
}
135+
136+
private static class BlackholeOutputStream extends OutputStream {
137+
138+
private final Blackhole blackhole;
139+
public long count = 0;
140+
141+
public BlackholeOutputStream(Blackhole blackhole) {
142+
this.blackhole = blackhole;
143+
}
144+
145+
@Override
146+
public void write(int b) {
147+
blackhole.consume(b);
148+
count++;
149+
}
150+
151+
@Override
152+
public void write(byte[] b) {
153+
write(b, 0, b.length);
154+
}
155+
156+
@Override
157+
public void write(byte[] b, int off, int len) {
158+
blackhole.consume(b);
159+
count += len;
160+
}
161+
162+
@Override
163+
public void flush() {
164+
165+
}
166+
167+
@Override
168+
public void close() {
169+
}
170+
}
171+
}
Lines changed: 40 additions & 56 deletions
Original file line numberDiff line numberDiff line change
@@ -1,77 +1,25 @@
11
package com.clickhouse.benchmark.clients;
22

3-
import com.clickhouse.benchmark.data.DataSet;
4-
import com.clickhouse.client.ClickHouseConfig;
5-
import com.clickhouse.client.api.Client;
63
import com.clickhouse.client.api.data_formats.RowBinaryFormatWriter;
7-
import com.clickhouse.client.api.data_formats.RowBinaryWithNamesAndTypesFormatReader;
8-
import com.clickhouse.client.api.data_formats.internal.BinaryStreamReader;
9-
import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream;
10-
import com.clickhouse.client.api.query.QueryResponse;
11-
import com.clickhouse.client.api.query.QuerySettings;
12-
import com.clickhouse.client.config.ClickHouseClientOption;
13-
import com.clickhouse.data.ClickHouseColumn;
144
import com.clickhouse.data.ClickHouseDataProcessor;
155
import com.clickhouse.data.ClickHouseFormat;
16-
import com.clickhouse.data.ClickHouseInputStream;
176
import com.clickhouse.data.ClickHouseOutputStream;
187
import com.clickhouse.data.ClickHouseRecord;
198
import com.clickhouse.data.ClickHouseSerializer;
20-
import com.clickhouse.data.format.ClickHouseRowBinaryProcessor;
21-
import com.clickhouse.data.stream.Lz4OutputStream;
22-
import net.jpountz.lz4.LZ4Factory;
239
import org.openjdk.jmh.annotations.Benchmark;
24-
import org.openjdk.jmh.annotations.Level;
25-
import org.openjdk.jmh.annotations.Setup;
2610
import org.openjdk.jmh.infra.Blackhole;
2711
import org.slf4j.Logger;
2812
import org.slf4j.LoggerFactory;
2913

30-
import java.io.ByteArrayInputStream;
31-
import java.io.ByteArrayOutputStream;
32-
import java.io.InputStream;
3314
import java.io.OutputStream;
34-
import java.nio.ByteBuffer;
35-
import java.util.Collections;
3615
import java.util.List;
37-
import java.util.Map;
3816

3917
public class Serializers extends BenchmarkBase {
4018
private static final Logger LOGGER = LoggerFactory.getLogger(Serializers.class);
4119

42-
private OutputStream createEmptyOutputStream() {
43-
return new OutputStream() {
44-
private long count = 0;
45-
46-
@Override
47-
public void write(int b) {
48-
count++;
49-
}
50-
51-
@Override
52-
public void write(byte[] b) {
53-
count += b.length;
54-
}
55-
56-
@Override
57-
public void write(byte[] b, int off, int len) {
58-
count += len;
59-
}
60-
61-
@Override
62-
public void flush() {
63-
64-
}
65-
66-
@Override
67-
public void close() {
68-
}
69-
};
70-
}
71-
7220
@Benchmark
73-
public void SerializerOutputStreamV1(DataState dataState) {
74-
OutputStream empty = createEmptyOutputStream();
21+
public void SerializerOutputStreamV1(DataState dataState, Blackhole blackhole) {
22+
OutputStream empty = new BlackholeOutputStream(blackhole);
7523
try {
7624
ClickHouseOutputStream chos = ClickHouseOutputStream.of(empty);
7725
ClickHouseDataProcessor p = dataState.dataSet.getClickHouseDataProcessor();
@@ -88,8 +36,8 @@ public void SerializerOutputStreamV1(DataState dataState) {
8836
}
8937

9038
@Benchmark
91-
public void SerializerOutputStreamV2(DataState dataState) {
92-
OutputStream empty = createEmptyOutputStream();
39+
public void SerializerOutputStreamV2(DataState dataState, Blackhole blackhole) {
40+
OutputStream empty = new BlackholeOutputStream(blackhole);
9341
try {
9442
RowBinaryFormatWriter w = new RowBinaryFormatWriter(empty, dataState.dataSet.getSchema(), ClickHouseFormat.RowBinary);
9543
for (List<Object> row : dataState.dataSet.getRowsOrdered()) {
@@ -105,4 +53,40 @@ public void SerializerOutputStreamV2(DataState dataState) {
10553
LOGGER.error("Error: ", e);
10654
}
10755
}
56+
57+
private static class BlackholeOutputStream extends OutputStream {
58+
59+
private final Blackhole blackhole;
60+
public long count = 0;
61+
62+
public BlackholeOutputStream(Blackhole blackhole) {
63+
this.blackhole = blackhole;
64+
}
65+
66+
@Override
67+
public void write(int b) {
68+
blackhole.consume(b);
69+
count++;
70+
}
71+
72+
@Override
73+
public void write(byte[] b) {
74+
write(b, 0, b.length);
75+
}
76+
77+
@Override
78+
public void write(byte[] b, int off, int len) {
79+
blackhole.consume(b);
80+
count += len;
81+
}
82+
83+
@Override
84+
public void flush() {
85+
86+
}
87+
88+
@Override
89+
public void close() {
90+
}
91+
}
10892
}
Lines changed: 50 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,50 @@
1+
package com.clickhouse.benchmark.data;
2+
3+
import com.clickhouse.data.format.BinaryStreamUtils;
4+
5+
import java.io.ByteArrayOutputStream;
6+
import java.time.LocalDateTime;
7+
import java.util.TimeZone;
8+
9+
public class SyntheticDataSet {
10+
11+
private final int capacity;
12+
13+
public SyntheticDataSet(int capacity) {
14+
this.capacity = capacity;
15+
generateData();
16+
}
17+
18+
private void generateData() {
19+
generateDateTimeValues();
20+
}
21+
22+
private void generateDateTimeValues() {
23+
ByteArrayOutputStream out = new ByteArrayOutputStream();
24+
dateTimeValues = new LocalDateTime[capacity];
25+
TimeZone tz = TimeZone.getTimeZone("UTC");
26+
27+
try {
28+
for (int i = 0; i < capacity; i++) {
29+
dateTimeValues[i] = LocalDateTime.now().plusSeconds(i);
30+
BinaryStreamUtils.writeDateTime64(out, dateTimeValues[i], 3, tz);
31+
32+
}
33+
} catch (Exception e) {
34+
throw new RuntimeException("Failed to generate date time values", e);
35+
}
36+
dateTimeValuesRowBinary = out.toByteArray();
37+
}
38+
39+
private LocalDateTime[] dateTimeValues;
40+
41+
private byte[] dateTimeValuesRowBinary;
42+
43+
public LocalDateTime[] getDateTimeValues() {
44+
return dateTimeValues;
45+
}
46+
47+
public byte[] getDateTimeValuesRowBinaryStream() {
48+
return dateTimeValuesRowBinary;
49+
}
50+
}

0 commit comments

Comments
 (0)