Skip to content

Commit 6377df1

Browse files
committed
Removed unnecessary object creation. Added executor pool configuration.
1 parent 25af97d commit 6377df1

File tree

18 files changed

+198
-72
lines changed

18 files changed

+198
-72
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/AbstractClient.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@
77
import java.util.concurrent.CompletableFuture;
88
import java.util.concurrent.CompletionException;
99
import java.util.concurrent.ExecutorService;
10+
import java.util.concurrent.TimeUnit;
1011
import java.util.concurrent.locks.ReadWriteLock;
1112
import java.util.concurrent.locks.ReentrantReadWriteLock;
1213
import java.util.function.Function;
@@ -252,7 +253,7 @@ public void init(ClickHouseConfig config) {
252253
if (this.executor == null) { // only initialize once
253254
int threads = config.getMaxThreadsPerClient();
254255
this.executor = threads < 1 ? ClickHouseClient.getExecutorService()
255-
: ClickHouseUtils.newThreadPool(this, threads, config.getMaxQueuedRequests());
256+
: ClickHouseUtils.newThreadPool(this, threads, threads, config.getMaxQueuedRequests(), TimeUnit.MINUTES.toMillis(10), true);
256257
}
257258

258259
initialized = true;

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -955,7 +955,6 @@ default boolean ping(ClickHouseNode server, int timeout) {
955955
.option(ClickHouseClientOption.ASYNC, false) // use current thread
956956
.option(ClickHouseClientOption.CONNECTION_TIMEOUT, timeout)
957957
.option(ClickHouseClientOption.SOCKET_TIMEOUT, timeout)
958-
.option(ClickHouseClientOption.BUFFER_SIZE, 8) // actually 4 bytes should be enough
959958
.option(ClickHouseClientOption.MAX_QUEUED_BUFFERS, 1) // enough with only one buffer
960959
.format(ClickHouseFormat.TabSeparated)
961960
.query("SELECT 1 FORMAT TabSeparated").execute()

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseSimpleResponse.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,10 @@
1212
import java.util.Collections;
1313
import java.util.LinkedList;
1414
import java.util.List;
15+
import java.util.Map;
1516
import java.util.TimeZone;
17+
import java.util.stream.Collectors;
18+
import java.util.stream.IntStream;
1619

1720
/**
1821
* A simple response built on top of two lists: columns and records.
@@ -104,6 +107,9 @@ public static ClickHouseResponse of(ClickHouseResponse response, ClickHouseRecor
104107
}
105108

106109
List<ClickHouseColumn> columns = response.getColumns();
110+
Map<String, Integer> columnIndex = IntStream.range(0, columns.size()).boxed().
111+
collect(Collectors.toMap(i->columns.get(i).getColumnName() , i -> i));
112+
107113
int size = columns.size();
108114
List<ClickHouseRecord> records = new LinkedList<>();
109115
int rowIndex = 0;
@@ -113,7 +119,7 @@ public static ClickHouseResponse of(ClickHouseResponse response, ClickHouseRecor
113119
values[i] = r.getValue(i).copy();
114120
}
115121

116-
ClickHouseRecord rec = ClickHouseSimpleRecord.of(columns, values);
122+
ClickHouseRecord rec = ClickHouseSimpleRecord.of(columnIndex, values);
117123
if (func != null) {
118124
func.update(rowIndex, rec);
119125
}
@@ -141,11 +147,12 @@ protected ClickHouseSimpleResponse(List<ClickHouseColumn> columns, List<ClickHou
141147
protected ClickHouseSimpleResponse(List<ClickHouseColumn> columns, ClickHouseValue[][] values,
142148
ClickHouseResponseSummary summary, TimeZone timeZone) {
143149
this.columns = columns;
144-
150+
Map<String, Integer> columnIndex = IntStream.range(0, columns.size()).boxed().
151+
collect(Collectors.toMap(i->columns.get(i).getColumnName() , i -> i));
145152
int len = values.length;
146153
List<ClickHouseRecord> list = new ArrayList<>(len);
147154
for (int i = 0; i < len; i++) {
148-
list.add(ClickHouseSimpleRecord.of(columns, values[i]));
155+
list.add(ClickHouseSimpleRecord.of(columnIndex, values[i]));
149156
}
150157

151158
this.records = Collections.unmodifiableList(list);

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

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,8 @@
1414
import java.util.List;
1515
import java.util.Map;
1616
import java.util.NoSuchElementException;
17+
import java.util.stream.Collectors;
18+
import java.util.stream.IntStream;
1719

1820
/**
1921
* This defines a data processor for dealing with serialization and
@@ -35,6 +37,7 @@ protected static final class DefaultSerDe {
3537
private final ClickHouseRecord currentRecord;
3638
private final Iterator<ClickHouseRecord> records;
3739
private final Iterator<ClickHouseValue> values;
40+
private final Map<String, Integer> columnsIndex;
3841

3942
DefaultSerDe(ClickHouseDataProcessor processor) throws IOException {
4043
if (processor.initialSettings == null || processor.initialSettings.isEmpty()) {
@@ -65,6 +68,7 @@ protected static final class DefaultSerDe {
6568
}
6669
}
6770
this.columnList = Collections.unmodifiableList(Arrays.asList(this.columns));
71+
this.columnsIndex = IntStream.range(0, columnList.size()).boxed().collect(Collectors.toMap(i->columnList.get(i).getColumnName() , i -> i));
6872

6973
if (processor.input == null) {
7074
this.currentRecord = ClickHouseRecord.EMPTY;
@@ -78,7 +82,7 @@ protected static final class DefaultSerDe {
7882
this.serializers[i] = processor.getSerializer(processor.config, this.columns[i]);
7983
}
8084
} else {
81-
this.currentRecord = new ClickHouseSimpleRecord(this.columnList, this.templates);
85+
this.currentRecord = new ClickHouseSimpleRecord(this.columnsIndex, this.templates);
8286

8387
this.records = ClickHouseChecker.nonNull(processor.initRecords(), "Records");
8488
this.values = ClickHouseChecker.nonNull(processor.initValues(), "Values");
Lines changed: 12 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -1,51 +1,44 @@
11
package com.clickhouse.data;
22

33
import java.util.Collections;
4-
import java.util.HashMap;
5-
import java.util.List;
64
import java.util.Map;
75

86
/**
97
* Default implementation of {@link com.clickhouse.data.ClickHouseRecord},
108
* which is simply a combination of list of columns and array of values.
119
*/
1210
public class ClickHouseSimpleRecord implements ClickHouseRecord {
13-
public static final ClickHouseSimpleRecord EMPTY = new ClickHouseSimpleRecord(Collections.emptyList(),
11+
public static final ClickHouseSimpleRecord EMPTY = new ClickHouseSimpleRecord(Collections.emptyMap(),
1412
new ClickHouseValue[0]);
1513

16-
private final List<ClickHouseColumn> columns;
14+
private final Map<String, Integer> columnsIndex;
1715
private ClickHouseValue[] values;
18-
private Map<String, Integer> columnsIndexes = null;
1916

2017
/**
2118
* Creates a record object to wrap given values.
2219
*
23-
* @param columns non-null list of columns
20+
* @param columnsIndex index of columns ord numbers
2421
* @param values non-null array of values
2522
* @return record
2623
*/
27-
public static ClickHouseRecord of(List<ClickHouseColumn> columns, ClickHouseValue[] values) {
28-
if (columns == null || values == null) {
24+
public static ClickHouseRecord of(Map<String, Integer> columnsIndex, ClickHouseValue[] values) {
25+
if (columnsIndex == null || values == null) {
2926
throw new IllegalArgumentException("Non-null columns and values are required");
30-
} else if (columns.size() != values.length) {
27+
} else if (columnsIndex.size() != values.length) {
3128
throw new IllegalArgumentException(ClickHouseUtils.format(
32-
"Mismatched count: we have %d columns but we got %d values", columns.size(), values.length));
29+
"Mismatched count: we have %d columns but we got %d values", columnsIndex.size(), values.length));
3330
} else if (values.length == 0) {
3431
return EMPTY;
3532
}
3633

37-
return new ClickHouseSimpleRecord(columns, values);
34+
return new ClickHouseSimpleRecord(columnsIndex, values);
3835
}
3936

40-
protected ClickHouseSimpleRecord(List<ClickHouseColumn> columns, ClickHouseValue[] values) {
41-
this.columns = columns;
37+
protected ClickHouseSimpleRecord(Map<String, Integer> columnsIndex, ClickHouseValue[] values) {
38+
this.columnsIndex = columnsIndex;
4239
this.values = values;
4340
}
4441

45-
protected List<ClickHouseColumn> getColumns() {
46-
return columns;
47-
}
48-
4942
protected ClickHouseValue[] getValues() {
5043
return values;
5144
}
@@ -76,7 +69,7 @@ public ClickHouseRecord copy() {
7669
for (int i = 0; i < len; i++) {
7770
vals[i] = values[i].copy();
7871
}
79-
return new ClickHouseSimpleRecord(columns, vals);
72+
return new ClickHouseSimpleRecord(columnsIndex, vals);
8073
}
8174

8275
@Override
@@ -86,25 +79,11 @@ public ClickHouseValue getValue(int index) {
8679

8780
@Override
8881
public ClickHouseValue getValue(String name) {
89-
if(columnsIndexes == null)
90-
columnsIndexes = new HashMap<>(columns.size());
91-
92-
return getValue(columnsIndexes.computeIfAbsent(name, this::computeColumnIndex));
82+
return getValue(columnsIndex.get(name));
9383
}
9484

9585
@Override
9686
public int size() {
9787
return values.length;
9888
}
99-
100-
private int computeColumnIndex(String name) {
101-
int index = 0;
102-
for (ClickHouseColumn c : columns) {
103-
if (c.getColumnName().equalsIgnoreCase(name)) {
104-
return index;
105-
}
106-
index++;
107-
}
108-
throw new IllegalArgumentException(ClickHouseUtils.format("Unable to find column [%s]", name));
109-
}
11089
}

clickhouse-data/src/test/java/com/clickhouse/data/ClickHouseSimpleRecordTest.java

Lines changed: 20 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -2,43 +2,51 @@
22

33
import java.util.Arrays;
44
import java.util.Collections;
5+
import java.util.List;
6+
import java.util.Map;
7+
import java.util.stream.Collectors;
8+
import java.util.stream.IntStream;
59

610
import com.clickhouse.data.value.ClickHouseLongValue;
711
import com.clickhouse.data.value.ClickHouseStringValue;
812

913
import org.testng.Assert;
1014
import org.testng.annotations.Test;
1115

16+
import javax.swing.*;
17+
1218
public class ClickHouseSimpleRecordTest {
1319
@Test(groups = { "unit" })
1420
public void testNullInput() {
1521
Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseSimpleRecord.of(null, null));
1622
Assert.assertThrows(IllegalArgumentException.class,
1723
() -> ClickHouseSimpleRecord.of(null, new ClickHouseValue[0]));
1824
Assert.assertThrows(IllegalArgumentException.class,
19-
() -> ClickHouseSimpleRecord.of(Collections.emptyList(), null));
25+
() -> ClickHouseSimpleRecord.of(Collections.emptyMap(), null));
2026

2127
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(null, null);
22-
Assert.assertNull(record.getColumns());
2328
Assert.assertNull(record.getValues());
2429
}
2530

2631
@Test(groups = { "unit" })
2732
public void testMismatchedColumnsAndValues() {
33+
2834
Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseSimpleRecord
29-
.of(Arrays.asList(ClickHouseColumn.of("a", "String")), new ClickHouseValue[0]));
35+
.of(Map.of("a", 0), new ClickHouseValue[0]));
3036

31-
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(Arrays.asList(ClickHouseColumn.of("a", "String")),
37+
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(Map.of("a", 0),
3238
new ClickHouseValue[0]);
33-
Assert.assertEquals(record.getColumns(), Arrays.asList(ClickHouseColumn.of("a", "String")));
3439
Assert.assertEquals(record.getValues(), new ClickHouseValue[0]);
3540
}
3641

3742
@Test(groups = { "unit" })
3843
public void testGetValueByIndex() {
39-
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(ClickHouseColumn.parse("a String, b UInt32"),
44+
List<ClickHouseColumn> columnList = ClickHouseColumn.parse("a String, b UInt32");
45+
Map<String, Integer> columnIndex = IntStream.range(0, columnList.size()).boxed().collect(Collectors.toMap(i->columnList.get(i).getColumnName() , i -> i));
46+
47+
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(columnIndex,
4048
new ClickHouseValue[] { ClickHouseStringValue.of("123"), ClickHouseLongValue.of(1L, true) });
41-
Assert.assertEquals(record.getColumns(), ClickHouseColumn.parse("a String, b UInt32"));
49+
// Assert.assertEquals(record.getColumns(), ClickHouseColumn.parse("a String, b UInt32"));
4250
Assert.assertEquals(record.getValues(),
4351
new ClickHouseValue[] { ClickHouseStringValue.of("123"), ClickHouseLongValue.of(1L, true) });
4452

@@ -60,12 +68,14 @@ public void testGetValueByIndex() {
6068

6169
@Test(groups = { "unit" })
6270
public void testGetValueByName() {
71+
List<ClickHouseColumn> columnList = ClickHouseColumn.parse("`a One` String, `x木哈哈x` UInt32, test Nullable(String)");
72+
Map<String, Integer> columnIndex = IntStream.range(0, columnList.size()).boxed().collect(Collectors.toMap(i->columnList.get(i).getColumnName() , i -> i));
6373
ClickHouseSimpleRecord record = new ClickHouseSimpleRecord(
64-
ClickHouseColumn.parse("`a One` String, `x木哈哈x` UInt32, test Nullable(String)"),
74+
columnIndex,
6575
new ClickHouseValue[] { ClickHouseStringValue.of("123"), ClickHouseLongValue.of(1L, true),
6676
ClickHouseStringValue.ofNull() });
67-
Assert.assertEquals(record.getColumns(),
68-
ClickHouseColumn.parse("`a One` String, `x木哈哈x` UInt32, test Nullable(String)"));
77+
// Assert.assertEquals(record.getColumns(),
78+
// ClickHouseColumn.parse("`a One` String, `x木哈哈x` UInt32, test Nullable(String)"));
6979
Assert.assertEquals(record.getValues(), new ClickHouseValue[] { ClickHouseStringValue.of("123"),
7080
ClickHouseLongValue.of(1L, true), ClickHouseStringValue.ofNull() });
7181

clickhouse-data/src/test/java/com/clickhouse/data/mapper/CustomRecordMappersTest.java

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

3+
import java.util.HashMap;
34
import java.util.List;
5+
import java.util.Map;
6+
import java.util.stream.IntStream;
47

8+
import org.apache.commons.lang3.stream.IntStreams;
59
import org.testng.Assert;
610
import org.testng.annotations.Test;
711

@@ -50,7 +54,9 @@ static class AnotherPojo extends ComplexPojo {
5054
public void testCustomConstructor() throws Exception {
5155
ClickHouseDataConfig config = new ClickHouseTestDataConfig();
5256
List<ClickHouseColumn> c = ClickHouseColumn.parse("id UInt32, name String");
53-
ClickHouseRecord r = ClickHouseSimpleRecord.of(c,
57+
HashMap<String, Integer> columnsIndex = IntStream.range(0, c.size()).boxed()
58+
.collect(HashMap::new, (m, i) -> m.put(c.get(i).getColumnName(), i), HashMap::putAll);
59+
ClickHouseRecord r = ClickHouseSimpleRecord.of(columnsIndex,
5460
new ClickHouseValue[] { ClickHouseStringValue.ofNull(), ClickHouseStringValue.ofNull() });
5561
Assert.assertThrows(IllegalArgumentException.class,
5662
() -> new CustomRecordMappers.RecordConstructor(Object.class, null).mapTo(r, null));
@@ -68,7 +74,10 @@ public void testCustomConstructor() throws Exception {
6874
public void testCustomCreator() throws Exception {
6975
ClickHouseDataConfig config = new ClickHouseTestDataConfig();
7076
List<ClickHouseColumn> c = ClickHouseColumn.parse("id UInt32, name String");
71-
ClickHouseRecord r = ClickHouseSimpleRecord.of(c,
77+
HashMap<String, Integer> columnsIndex = IntStream.range(0, c.size()).boxed()
78+
.collect(HashMap::new, (m, i) -> m.put(c.get(i).getColumnName(), i), HashMap::putAll);
79+
80+
ClickHouseRecord r = ClickHouseSimpleRecord.of(columnsIndex,
7281
new ClickHouseValue[] { ClickHouseStringValue.ofNull(), ClickHouseStringValue.ofNull() });
7382
Assert.assertThrows(IllegalArgumentException.class,
7483
() -> new CustomRecordMappers.RecordCreator(Object.class, null).mapTo(r, null));

clickhouse-data/src/test/java/com/clickhouse/data/mapper/DynamicRecordMapperTest.java

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

3+
import java.util.HashMap;
34
import java.util.List;
5+
import java.util.stream.IntStream;
46

57
import org.testng.Assert;
68
import org.testng.annotations.Test;
@@ -109,14 +111,17 @@ public void testSimplePojo() {
109111
// Assert.assertThrows(IllegalArgumentException.class,
110112
// () -> mapper.mapTo(ClickHouseSimpleRecord.of(columns, values),
111113
// PrivatePojo.class));
112-
SimplePojo pojo = mapper.mapTo(ClickHouseSimpleRecord.of(columns, values), SimplePojo.class, new SimplePojo());
114+
HashMap<String, Integer> columnsIndex = IntStream.range(0, columns.size()).boxed()
115+
.collect(HashMap::new, (m, i) -> m.put(columns.get(i).getColumnName(), i), HashMap::putAll);
116+
117+
SimplePojo pojo = mapper.mapTo(ClickHouseSimpleRecord.of(columnsIndex, values), SimplePojo.class, new SimplePojo());
113118
Assert.assertNotNull(pojo, "Result should NOT be null");
114119
Assert.assertEquals(pojo.getId(), values[1].asInteger());
115120
Assert.assertEquals(pojo.getName(), "");
116121
Assert.assertEquals(pojo.getDescription(), values[2].asString());
117122

118123
RecordPojo rpojo = RecordMapperFactory.of(config, columns, RecordPojo.class)
119-
.mapTo(ClickHouseSimpleRecord.of(columns, values), RecordPojo.class);
124+
.mapTo(ClickHouseSimpleRecord.of(columnsIndex, values), RecordPojo.class);
120125
Assert.assertNotNull(rpojo);
121126
Assert.assertEquals(rpojo.getId(), values[1].asInteger());
122127
Assert.assertEquals(rpojo.getName(), values[0].asString());

clickhouse-data/src/test/java/com/clickhouse/data/mapper/RecordMapperFactoryTest.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,8 @@
11
package com.clickhouse.data.mapper;
22

3+
import java.util.HashMap;
34
import java.util.List;
5+
import java.util.stream.IntStream;
46

57
import org.testng.Assert;
68
import org.testng.annotations.Test;
@@ -54,7 +56,10 @@ public void testGet() {
5456
public void testGetCustom() {
5557
ClickHouseDataConfig config = new ClickHouseTestDataConfig();
5658
List<ClickHouseColumn> columns = ClickHouseColumn.parse("id UInt64, str Nullable(String)");
57-
ClickHouseRecord r = ClickHouseSimpleRecord.of(columns,
59+
HashMap<String, Integer> columnsIndex = IntStream.range(0, columns.size()).boxed()
60+
.collect(HashMap::new, (m, i) -> m.put(columns.get(i).getColumnName(), i), HashMap::putAll);
61+
62+
ClickHouseRecord r = ClickHouseSimpleRecord.of(columnsIndex,
5863
new ClickHouseValue[] { ClickHouseLongValue.ofUnsigned(5), ClickHouseStringValue.of("555...") });
5964
ClickHouseRecordMapper mapper = RecordMapperFactory.of(config, columns, Object.class);
6065
Assert.assertNotNull(mapper);

0 commit comments

Comments
 (0)