|
1 | 1 | package com.clickhouse.benchmark.clients; |
2 | 2 |
|
3 | 3 | import com.clickhouse.benchmark.data.DataSet; |
| 4 | +import com.clickhouse.client.api.data_formats.RowBinaryFormatWriter; |
| 5 | +import com.clickhouse.client.api.insert.InsertResponse; |
4 | 6 | import com.clickhouse.client.api.internal.ClickHouseLZ4OutputStream; |
| 7 | +import com.clickhouse.data.ClickHouseColumn; |
| 8 | +import com.clickhouse.data.ClickHouseDataProcessor; |
| 9 | +import com.clickhouse.data.ClickHouseFormat; |
5 | 10 | import com.clickhouse.data.ClickHouseOutputStream; |
| 11 | +import com.clickhouse.data.ClickHousePassThruStream; |
| 12 | +import com.clickhouse.data.ClickHouseRecord; |
| 13 | +import com.clickhouse.data.ClickHouseSerializer; |
6 | 14 | import com.clickhouse.data.stream.Lz4OutputStream; |
7 | 15 | import net.jpountz.lz4.LZ4Factory; |
8 | 16 | import org.openjdk.jmh.annotations.Benchmark; |
9 | 17 | import org.slf4j.Logger; |
10 | 18 | import org.slf4j.LoggerFactory; |
11 | 19 |
|
12 | 20 | import java.io.ByteArrayOutputStream; |
| 21 | +import java.io.OutputStream; |
| 22 | +import java.util.List; |
| 23 | +import java.util.Map; |
13 | 24 |
|
14 | 25 | public class Components extends BenchmarkBase { |
15 | 26 | private static final Logger LOGGER = LoggerFactory.getLogger(Components.class); |
@@ -42,4 +53,70 @@ public void CompressingOutputStreamV2(DataState dataState) { |
42 | 53 | LOGGER.error("Error: ", e); |
43 | 54 | } |
44 | 55 | } |
| 56 | + |
| 57 | + private OutputStream createEmptyOutputStream() { |
| 58 | + return new OutputStream() { |
| 59 | + private long count = 0; |
| 60 | + |
| 61 | + @Override |
| 62 | + public void write(int b) { |
| 63 | + count++; |
| 64 | + } |
| 65 | + |
| 66 | + @Override |
| 67 | + public void write(byte[] b) { |
| 68 | + count += b.length; |
| 69 | + } |
| 70 | + |
| 71 | + @Override |
| 72 | + public void write(byte[] b, int off, int len) { |
| 73 | + count += len; |
| 74 | + } |
| 75 | + |
| 76 | + @Override |
| 77 | + public void flush() { |
| 78 | + |
| 79 | + } |
| 80 | + |
| 81 | + @Override |
| 82 | + public void close() { |
| 83 | + } |
| 84 | + }; |
| 85 | + } |
| 86 | + @Benchmark |
| 87 | + public void SerializerOutputStreamV1(DataState dataState) { |
| 88 | + OutputStream empty = createEmptyOutputStream(); |
| 89 | + |
| 90 | + try { |
| 91 | + ClickHouseOutputStream chos = ClickHouseOutputStream.of(empty); |
| 92 | + ClickHouseDataProcessor p = dataState.dataSet.getClickHouseDataProcessor(); |
| 93 | + ClickHouseSerializer[] serializers = p.getSerializers(getClientV1().getConfig(), p.getColumns()); |
| 94 | + for (ClickHouseRecord record : dataState.dataSet.getClickHouseRecords()) { |
| 95 | + for (int i = 0; i < serializers.length; i++) { |
| 96 | + serializers[i].serialize(record.getValue(i), chos); |
| 97 | + } |
| 98 | + } |
| 99 | + chos.flush(); |
| 100 | + } catch (Exception e) { |
| 101 | + LOGGER.error("Error: ", e); |
| 102 | + } |
| 103 | + } |
| 104 | + |
| 105 | + @Benchmark |
| 106 | + public void SerializerOutputStreamV2(DataState dataState) { |
| 107 | + OutputStream empty = createEmptyOutputStream(); |
| 108 | + try { |
| 109 | + RowBinaryFormatWriter w = new RowBinaryFormatWriter(empty, dataState.dataSet.getSchema(), ClickHouseFormat.RowBinary); |
| 110 | + List<ClickHouseColumn> columns = dataState.dataSet.getSchema().getColumns(); |
| 111 | + for (Map<String, Object> row : dataState.dataSet.getRows()) { |
| 112 | + for (ClickHouseColumn column : columns) { |
| 113 | + w.setValue(column.getColumnName(),row.get(column.getColumnName())); |
| 114 | + } |
| 115 | + w.commitRow(); |
| 116 | + } |
| 117 | + empty.flush(); |
| 118 | + } catch (Exception e) { |
| 119 | + LOGGER.error("Error: ", e); |
| 120 | + } |
| 121 | + } |
45 | 122 | } |
0 commit comments