Skip to content

Commit fdb8205

Browse files
authored
Merge pull request #1613 from ClickHouse/feature_query_settings
Added all query settings and implemented simple validation
2 parents cb6a298 + e5d453c commit fdb8205

File tree

9 files changed

+679
-77
lines changed

9 files changed

+679
-77
lines changed

client-v2/pom.xml

Lines changed: 17 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -27,20 +27,25 @@
2727
<artifactId>clickhouse-http-client</artifactId>
2828
<version>${revision}</version>
2929
</dependency>
30+
<dependency>
31+
<groupId>org.slf4j</groupId>
32+
<artifactId>slf4j-api</artifactId>
33+
<version>${slf4j.version}</version>
34+
</dependency>
3035
<dependency>
3136
<groupId>org.apache.httpcomponents.client5</groupId>
3237
<artifactId>httpclient5</artifactId>
33-
<version>5.2.1</version>
38+
<version>${apache.httpclient.version}</version>
3439
</dependency>
3540
<dependency>
3641
<groupId>org.apache.httpcomponents.core5</groupId>
3742
<artifactId>httpcore5</artifactId>
38-
<version>5.2.1</version>
43+
<version>${apache.httpclient.version}</version>
3944
</dependency>
4045
<dependency>
4146
<groupId>org.apache.httpcomponents.core5</groupId>
4247
<artifactId>httpcore5-h2</artifactId>
43-
<version>5.2.1</version>
48+
<version>${apache.httpclient.version}</version>
4449
</dependency>
4550
<dependency>
4651
<groupId>com.github.luben</groupId>
@@ -76,6 +81,7 @@
7681
<scope>compile</scope>
7782
</dependency>
7883

84+
<!-- Test dependencies -->
7985
<dependency>
8086
<groupId>${project.parent.groupId}</groupId>
8187
<artifactId>clickhouse-client</artifactId>
@@ -86,11 +92,13 @@
8692
<dependency>
8793
<groupId>org.slf4j</groupId>
8894
<artifactId>slf4j-simple</artifactId>
95+
<version>${slf4j.version}</version>
8996
<scope>test</scope>
9097
</dependency>
9198
<dependency>
9299
<groupId>org.testcontainers</groupId>
93100
<artifactId>testcontainers</artifactId>
101+
<version>${testcontainers.version}</version>
94102
<scope>test</scope>
95103
</dependency>
96104
<dependency>
@@ -101,13 +109,15 @@
101109
<dependency>
102110
<groupId>org.testng</groupId>
103111
<artifactId>testng</artifactId>
112+
<version>${testng.version}</version>
104113
<scope>test</scope>
105114
</dependency>
106115
<dependency>
107-
<groupId>org.projectlombok</groupId>
108-
<artifactId>lombok</artifactId>
109-
<version>1.18.32</version>
110-
<scope>provided</scope>
116+
<groupId>${project.parent.groupId}</groupId>
117+
<artifactId>clickhouse-client</artifactId>
118+
<version>${revision}</version>
119+
<type>test-jar</type>
120+
<scope>test</scope>
111121
</dependency>
112122
</dependencies>
113123

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

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

33
import com.clickhouse.client.*;
4+
import com.clickhouse.client.api.internal.SettingsConverter;
5+
import com.clickhouse.client.api.internal.ValidationUtils;
46
import com.clickhouse.data.ClickHouseColumn;
57

68
import java.io.InputStream;
@@ -13,6 +15,10 @@
1315
import com.clickhouse.client.api.query.QueryResponse;
1416
import com.clickhouse.client.api.query.QuerySettings;
1517
import com.clickhouse.data.ClickHouseFormat;
18+
import org.slf4j.Logger;
19+
import org.slf4j.LoggerFactory;
20+
import org.slf4j.MDC;
21+
import org.slf4j.helpers.BasicMDCAdapter;
1622

1723
import java.util.concurrent.CompletableFuture;
1824
import java.util.concurrent.Future;
@@ -24,6 +30,8 @@ public class Client {
2430
private Set<String> endpoints;
2531
private Map<String, String> configuration;
2632
private List<ClickHouseNode> serverNodes = new ArrayList<>();
33+
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
34+
2735
private Client(Set<String> endpoints, Map<String,String> configuration) {
2836
this.endpoints = endpoints;
2937
this.configuration = configuration;
@@ -189,16 +197,22 @@ public Future<InsertResponse> insert(String tableName,
189197
* @return
190198
*/
191199
public Future<QueryResponse> query(String sqlQuery, Map<String, Object> qparams, QuerySettings settings) {
192-
ClickHouseClient clientQuery = ClickHouseClient.newInstance(ClickHouseProtocol.HTTP);
193-
ClickHouseRequest request = clientQuery.read(getServerNode());
200+
ClickHouseClient client = createClient();
201+
ClickHouseRequest<?> request = client.read(getServerNode());
202+
request.options(SettingsConverter.toRequestOptions(settings.getAllSettings()));
203+
request.settings(SettingsConverter.toRequestSettings(settings.getAllSettings()));
194204
request.query(sqlQuery, settings.getQueryID());
195-
// TODO: convert qparams to map[string, string]
196-
request.params(qparams);
197-
return CompletableFuture.completedFuture(new QueryResponse(clientQuery.execute(request)));
205+
request.format(ClickHouseFormat.valueOf(settings.getFormat()));
206+
if (qparams != null && !qparams.isEmpty()) {
207+
request.params(qparams);
208+
}
209+
MDC.put("queryId", settings.getQueryID());
210+
LOG.debug("Executing request: {}", request);
211+
return CompletableFuture.completedFuture(new QueryResponse(client, request.execute()));
198212
}
199213

200214
public TableSchema getTableSchema(String table, String database) {
201-
try (ClickHouseClient clientQuery = ClickHouseClient.newInstance(ClickHouseProtocol.HTTP)) {
215+
try (ClickHouseClient clientQuery = createClient()) {
202216
ClickHouseRequest request = clientQuery.read(getServerNode());
203217
// XML - because java has a built-in XML parser. Will consider CSV later.
204218
request.query("DESCRIBE TABLE " + table + " FORMAT " + ClickHouseFormat.TSKV.name());
@@ -210,4 +224,39 @@ public TableSchema getTableSchema(String table, String database) {
210224
}
211225
}
212226
}
227+
228+
229+
private ClickHouseClient createClient() {
230+
ClickHouseConfig clientConfig = new ClickHouseConfig();
231+
return ClickHouseClient.builder().config(clientConfig)
232+
.nodeSelector(ClickHouseNodeSelector.of(ClickHouseProtocol.HTTP))
233+
.build();
234+
}
235+
236+
private static final Set<String> COMPRESS_ALGORITHMS = ValidationUtils.whiteList("LZ4", "LZ4HC", "ZSTD", "ZSTDHC", "NONE");
237+
238+
public static Set<String> getCompressAlgorithms() {
239+
return COMPRESS_ALGORITHMS;
240+
}
241+
242+
private static final Set<String> OUTPUT_FORMATS = createFormatWhitelist("output");
243+
244+
private static final Set<String> INPUT_FORMATS = createFormatWhitelist("input");
245+
246+
public static Set<String> getOutputFormats() {
247+
return OUTPUT_FORMATS;
248+
}
249+
250+
private static Set<String> createFormatWhitelist(String shouldSupport) {
251+
Set<String> formats = new HashSet<>();
252+
boolean supportOutput = "output".equals(shouldSupport);
253+
boolean supportInput = "input".equals(shouldSupport);
254+
boolean supportBoth = "both".equals(shouldSupport);
255+
for (ClickHouseFormat format : ClickHouseFormat.values()) {
256+
if ((supportOutput && format.supportsOutput()) || (supportInput && format.supportsInput()) || (supportBoth)) {
257+
formats.add(format.name());
258+
}
259+
}
260+
return Collections.unmodifiableSet(formats);
261+
}
213262
}
Lines changed: 159 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,159 @@
1+
package com.clickhouse.client.api.internal;
2+
3+
import com.clickhouse.client.config.ClickHouseClientOption;
4+
import com.clickhouse.client.http.config.ClickHouseHttpOption;
5+
import com.clickhouse.config.ClickHouseOption;
6+
7+
import java.io.Serializable;
8+
import java.util.Arrays;
9+
import java.util.Collection;
10+
import java.util.Collections;
11+
import java.util.HashMap;
12+
import java.util.HashSet;
13+
import java.util.Map;
14+
import java.util.Set;
15+
import java.util.regex.Pattern;
16+
17+
public class SettingsConverter {
18+
19+
public static Map<String, Serializable> toRequestSettings(Map<String, Object> settings) {
20+
Map<String, Serializable> requestSettings = new HashMap<>();
21+
22+
for (Map.Entry<String, Object> entry : settings.entrySet()) {
23+
if (!REQUEST_SETTINGS.contains(entry.getKey())) {
24+
continue;
25+
}
26+
27+
if (entry.getValue() instanceof Map<?,?>) {
28+
Map<String, String> map = (Map<String, String>) entry.getValue();
29+
requestSettings.put(entry.getKey(), convertMapToStringValue(map));
30+
} else if (entry.getValue() instanceof Collection<?>) {
31+
Collection<?> collection = (Collection<?>) entry.getValue();
32+
requestSettings.put(entry.getKey(), convertCollectionToStringValue(collection));
33+
} else {
34+
requestSettings.put(entry.getKey(), (Serializable) entry.getValue());
35+
}
36+
}
37+
38+
return requestSettings;
39+
}
40+
41+
public static Map<ClickHouseOption, Serializable> toRequestOptions(Map<String, Object> settings) {
42+
Map<ClickHouseOption, Serializable> requestOptions = new HashMap<>();
43+
44+
for (Map.Entry<String, Object> entry : settings.entrySet()) {
45+
if (!REQUEST_OPTIONS.containsKey(entry.getKey())) {
46+
continue;
47+
}
48+
49+
ClickHouseOption option = REQUEST_OPTIONS.get(entry.getKey());
50+
if (entry.getValue() instanceof Map<?,?>) {
51+
Map<String, String> map = (Map<String, String>) entry.getValue();
52+
requestOptions.put(option, convertMapToStringValue(map));
53+
} else if (entry.getValue() instanceof Collection<?>) {
54+
Collection<?> collection = (Collection<?>) entry.getValue();
55+
requestOptions.put(option, convertCollectionToStringValue(collection));
56+
} else {
57+
requestOptions.put(option, (Serializable) entry.getValue());
58+
}
59+
}
60+
61+
return requestOptions;
62+
}
63+
64+
private static String convertMapToStringValue(Map<String, String> map) {
65+
StringBuilder sb = new StringBuilder();
66+
for (Map.Entry<String, String> e : map.entrySet()) {
67+
sb.append(escape(e.getKey())).append('=').append(escape(e.getValue())).append(',');
68+
}
69+
sb.setLength(sb.length() - 1);
70+
return sb.toString();
71+
}
72+
73+
private static String convertCollectionToStringValue(Collection<?> collection) {
74+
StringBuilder sb = new StringBuilder();
75+
for (Object value : collection) {
76+
sb.append(escape(value.toString())).append(',');
77+
}
78+
sb.setLength(sb.length() - 1);
79+
return sb.toString();
80+
}
81+
private static final Pattern ESCAPE_PATTERN = Pattern.compile("[,'\\\"=\\t\\n]{1}");
82+
83+
public static String escape(String value) {
84+
return ESCAPE_PATTERN.matcher(value).replaceAll("\\\\$0");
85+
}
86+
87+
private static final Map<String, ClickHouseOption> REQUEST_OPTIONS = createMapOfRequestOptions();
88+
private static final Set<String> REQUEST_SETTINGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
89+
ClickHouseClientOption.MAX_EXECUTION_TIME.getKey(),
90+
ClickHouseClientOption.MAX_RESULT_ROWS.getKey(),
91+
"extremes",
92+
"role"
93+
)));
94+
95+
public static Map<String, ClickHouseOption> createMapOfRequestOptions() {
96+
Map<String, ClickHouseOption> map = new HashMap<>();
97+
98+
Arrays.asList(ClickHouseClientOption.FORMAT,
99+
ClickHouseClientOption.MAX_EXECUTION_TIME,
100+
ClickHouseHttpOption.CUSTOM_PARAMS,
101+
ClickHouseClientOption.AUTO_DISCOVERY,
102+
ClickHouseClientOption.CUSTOM_SETTINGS,
103+
ClickHouseClientOption.CUSTOM_SOCKET_FACTORY,
104+
ClickHouseClientOption.CUSTOM_SOCKET_FACTORY_OPTIONS,
105+
ClickHouseClientOption.CLIENT_NAME,
106+
ClickHouseClientOption.DECOMPRESS,
107+
ClickHouseClientOption.DECOMPRESS_ALGORITHM,
108+
ClickHouseClientOption.DECOMPRESS_LEVEL,
109+
ClickHouseClientOption.COMPRESS,
110+
ClickHouseClientOption.COMPRESS_ALGORITHM,
111+
ClickHouseClientOption.COMPRESS_LEVEL,
112+
ClickHouseClientOption.CONNECTION_TIMEOUT,
113+
ClickHouseClientOption.DATABASE,
114+
ClickHouseClientOption.MAX_BUFFER_SIZE,
115+
ClickHouseClientOption.BUFFER_SIZE,
116+
ClickHouseClientOption.BUFFER_QUEUE_VARIATION,
117+
ClickHouseClientOption.READ_BUFFER_SIZE,
118+
ClickHouseClientOption.WRITE_BUFFER_SIZE,
119+
ClickHouseClientOption.REQUEST_CHUNK_SIZE,
120+
ClickHouseClientOption.REQUEST_BUFFERING,
121+
ClickHouseClientOption.RESPONSE_BUFFERING,
122+
ClickHouseClientOption.MAX_MAPPER_CACHE,
123+
ClickHouseClientOption.MAX_QUEUED_BUFFERS,
124+
ClickHouseClientOption.MAX_QUEUED_REQUESTS,
125+
ClickHouseClientOption.MAX_RESULT_ROWS,
126+
ClickHouseClientOption.MAX_THREADS_PER_CLIENT,
127+
ClickHouseClientOption.PRODUCT_NAME,
128+
ClickHouseClientOption.NODE_CHECK_INTERVAL,
129+
ClickHouseClientOption.FAILOVER,
130+
ClickHouseClientOption.RETRY,
131+
ClickHouseClientOption.REPEAT_ON_SESSION_LOCK,
132+
ClickHouseClientOption.REUSE_VALUE_WRAPPER,
133+
ClickHouseClientOption.SERVER_TIME_ZONE,
134+
ClickHouseClientOption.SERVER_VERSION,
135+
ClickHouseClientOption.SESSION_TIMEOUT,
136+
ClickHouseClientOption.SESSION_CHECK,
137+
ClickHouseClientOption.SOCKET_TIMEOUT,
138+
ClickHouseClientOption.SSL,
139+
ClickHouseClientOption.SSL_MODE,
140+
ClickHouseClientOption.SSL_ROOT_CERTIFICATE,
141+
ClickHouseClientOption.SSL_CERTIFICATE,
142+
ClickHouseClientOption.SSL_KEY,
143+
ClickHouseClientOption.KEY_STORE_TYPE,
144+
ClickHouseClientOption.TRUST_STORE,
145+
ClickHouseClientOption.KEY_STORE_PASSWORD,
146+
ClickHouseClientOption.TRANSACTION_TIMEOUT,
147+
ClickHouseClientOption.WIDEN_UNSIGNED_TYPES,
148+
ClickHouseClientOption.USE_BINARY_STRING,
149+
ClickHouseClientOption.USE_BLOCKING_QUEUE,
150+
ClickHouseClientOption.USE_COMPILATION,
151+
ClickHouseClientOption.USE_OBJECTS_IN_ARRAYS,
152+
ClickHouseClientOption.USE_SERVER_TIME_ZONE,
153+
ClickHouseClientOption.USE_SERVER_TIME_ZONE_FOR_DATES,
154+
ClickHouseClientOption.USE_TIME_ZONE)
155+
.forEach(option -> map.put(option.getKey(), option));
156+
157+
return Collections.unmodifiableMap(map);
158+
}
159+
}
Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,60 @@
1+
package com.clickhouse.client.api.internal;
2+
3+
import java.util.Arrays;
4+
import java.util.Collections;
5+
import java.util.HashSet;
6+
import java.util.Set;
7+
8+
public class ValidationUtils {
9+
10+
public static void checkRange(int value, int min, int max, String name) {
11+
if (value < min || value > max) {
12+
throw new SettingsValidationException(name, "\"" + name + "\" must be in range [" + min + ", " + max + "]");
13+
}
14+
}
15+
16+
public static void checkPositive(int value, String name) {
17+
if (value <= 0) {
18+
throw new SettingsValidationException(name, "\"" + name + "\" must be positive");
19+
}
20+
}
21+
22+
public static void checkNonBlank(String value, String name) {
23+
if (value == null || value.isEmpty()) {
24+
throw new SettingsValidationException(name, "\"" + name + "\" must be non-null and non-empty");
25+
}
26+
}
27+
28+
public static void checkNotNull(Object value, String name) {
29+
if (value == null) {
30+
throw new SettingsValidationException(name, "\"" + name + "\" must be non-null");
31+
}
32+
}
33+
34+
public static void checkValueFromSet(Object value, String name, Set<?> validValues) {
35+
if (!validValues.contains(value)) {
36+
throw new SettingsValidationException(name, "\"" + name + "\" must be one of " + validValues);
37+
}
38+
}
39+
40+
/**
41+
* Creates a unmodifiable set from the given values.
42+
* @param values
43+
* @return
44+
* @param <T>
45+
*/
46+
public static <T> Set<T> whiteList(T... values) {
47+
return Collections.unmodifiableSet(new HashSet<>(Arrays.asList(values)));
48+
}
49+
50+
public static class SettingsValidationException extends IllegalArgumentException {
51+
private static final long serialVersionUID = 1L;
52+
53+
private final String key;
54+
55+
public SettingsValidationException(String key, String message) {
56+
super(message);
57+
this.key = key;
58+
}
59+
}
60+
}

0 commit comments

Comments
 (0)