-
Notifications
You must be signed in to change notification settings - Fork 624
Expand file tree
/
Copy pathQuerySettings.java
More file actions
354 lines (304 loc) · 11.2 KB
/
QuerySettings.java
File metadata and controls
354 lines (304 loc) · 11.2 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
package com.clickhouse.client.api.query;
import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.Session;
import com.clickhouse.client.api.internal.CommonSettings;
import com.clickhouse.client.api.internal.ServerSettings;
import com.clickhouse.client.api.internal.ValidationUtils;
import com.clickhouse.data.ClickHouseFormat;
import java.time.temporal.ChronoUnit;
import java.util.Collection;
import java.util.Map;
import java.util.TimeZone;
/**
* <p>Query settings class represents a set of settings that can be used to customize query execution.</p>
*/
public class QuerySettings {
public static final int MINIMAL_READ_BUFFER_SIZE = 8192;
private final CommonSettings settings;
public QuerySettings(Map<String, Object> settings) {
this.settings = new CommonSettings();
for (Map.Entry<String, Object> entry : settings.entrySet()) {
this.settings.setOption(entry.getKey(), entry.getValue());
}
}
public QuerySettings() {
this.settings = new CommonSettings();
}
private QuerySettings(CommonSettings settings) {
this.settings = settings;
}
/**
* Sets a configuration option. This method can be used to set any configuration option.
* There is no specific validation is done on the key or value.
*
* @param option - configuration option name
* @param value - configuration option value
*/
public QuerySettings setOption(String option, Object value) {
settings.setOption(option, value);
return this;
}
public QuerySettings resetOption(String option) {
settings.resetOption(option);
return this;
}
/**
* Gets a configuration option.
*
* @param option - configuration option name
* @return configuration option value
*/
public Object getOption(String option) {
return settings.getOption(option);
}
/**
* Get raw settings. Returns reference to internal map, so any changes will affect this object.
*
* @return all settings map
*/
public Map<String, Object> getAllSettings() {
return settings.getAllSettings();
}
/**
* Sets the query id. This id will be sent to the server and can be used to identify the query.
*/
public QuerySettings setQueryId(String queryId) {
settings.setQueryId(queryId);
return this;
}
public String getQueryId() {
return settings.getQueryId();
}
/**
* Sets ClickHouse session id for this operation.
*/
public QuerySettings setSessionId(String sessionId) {
settings.setSessionId(sessionId);
return this;
}
public String getSessionId() {
return settings.getSessionId();
}
/**
* Sets ClickHouse session check flag for this operation.
*/
public QuerySettings setSessionCheck(boolean sessionCheck) {
settings.setSessionCheck(sessionCheck);
return this;
}
public Boolean getSessionCheck() {
return settings.getSessionCheck();
}
/**
* Sets ClickHouse session timeout (seconds) for this operation.
*/
public QuerySettings setSessionTimeout(int timeoutInSeconds) {
settings.setSessionTimeout(timeoutInSeconds);
return this;
}
public Integer getSessionTimeout() {
return settings.getSessionTimeout();
}
/**
* Sets ClickHouse session timezone for this operation.
*/
public QuerySettings setSessionTimezone(String timezone) {
settings.setSessionTimezone(timezone);
return this;
}
public String getSessionTimezone() {
return settings.getSessionTimezone();
}
public QuerySettings use(Session session) {
settings.use(session);
return this;
}
public QuerySettings clearSession() {
settings.clearSession();
return this;
}
/**
* Read buffer is used for reading data from a server. Size is in bytes.
* Minimal value is {@value MINIMAL_READ_BUFFER_SIZE} bytes.
*/
public QuerySettings setReadBufferSize(Integer size) {
ValidationUtils.checkNotNull(size, "read_buffer_size");
ValidationUtils.checkRange(size, MINIMAL_READ_BUFFER_SIZE, Integer.MAX_VALUE, "read_buffer_size");
settings.setOption("read_buffer_size", size);
return this;
}
public Integer getReadBufferSize() {
return (Integer) settings.getOption("read_buffer_size");
}
/**
* Sets output format for a server response.
*/
public QuerySettings setFormat(ClickHouseFormat format) {
settings.setOption("format", format);
return this;
}
public ClickHouseFormat getFormat() {
return (ClickHouseFormat) settings.getOption("format");
}
/**
* Maximum query execution time in seconds on server. 0 means no limit.
* If query is not finished in this time then server will send an exception.
*/
public QuerySettings setMaxExecutionTime(Integer maxExecutionTime) {
serverSetting(ServerSettings.MAX_EXECUTION_TIME, String.valueOf(maxExecutionTime));
return this;
}
public Integer getMaxExecutionTime() {
String val = (String) settings.getOption(
ClientConfigProperties.serverSetting(ServerSettings.MAX_EXECUTION_TIME));
return val == null ? null : Integer.valueOf(val);
}
/**
* Sets database to be used for a request.
*/
public QuerySettings setDatabase(String database) {
settings.setDatabase(database);
return this;
}
public String getDatabase() {
return settings.getDatabase();
}
/**
* Requests the server to wait for the and of the query before sending response. Useful for getting accurate summary.
*/
public QuerySettings waitEndOfQuery(Boolean waitEndOfQuery) {
serverSetting(ServerSettings.WAIT_END_OF_QUERY, waitEndOfQuery ? "1" : "0");
return this;
}
public QuerySettings setUseServerTimeZone(Boolean useServerTimeZone) {
if (settings.hasOption(ClientConfigProperties.USE_TIMEZONE.getKey())) {
throw new ValidationUtils.SettingsValidationException(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey(),
"Cannot set both use_time_zone and use_server_time_zone");
}
settings.setOption(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey(), useServerTimeZone);
return this;
}
public Boolean getUseServerTimeZone() {
return (Boolean) settings.getOption(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey());
}
public QuerySettings setUseTimeZone(String timeZone) {
if (settings.hasOption(ClientConfigProperties.USE_SERVER_TIMEZONE.getKey())) {
throw new ValidationUtils.SettingsValidationException(ClientConfigProperties.USE_TIMEZONE.getKey(),
"Cannot set both use_time_zone and use_server_time_zone");
}
settings.setOption(ClientConfigProperties.USE_TIMEZONE.getKey(), TimeZone.getTimeZone(timeZone));
return this;
}
public TimeZone getServerTimeZone() {
return (TimeZone) settings.getOption(ClientConfigProperties.SERVER_TIMEZONE.getKey());
}
/**
* Defines list of headers that should be sent with current request. The Client will use a header value
* defined in {@code headers} instead of any other.
*
* @param key - header name.
* @param value - header value.
* @return same instance of the builder
* @see Client.Builder#httpHeaders(Map)
*/
public QuerySettings httpHeader(String key, String value) {
settings.httpHeader(key, value);
return this;
}
/**
* {@see #httpHeader(String, String)} but for multiple values.
*
* @param key - name of the header
* @param values - collection of values
* @return same instance of the builder
*/
public QuerySettings httpHeader(String key, Collection<String> values) {
settings.httpHeader(key, values);
return this;
}
/**
* {@see #httpHeader(String, String)} but for multiple headers.
*
* @param headers - map of headers
* @return same instance of the builder
*/
public QuerySettings httpHeaders(Map<String, String> headers) {
settings.httpHeaders(headers);
return this;
}
/**
* Defines list of server settings that should be sent with each request. The Client will use a setting value
* defined in {@code settings} instead of any other.
* Operation settings may override these values.
*
* @param name - name of the setting
* @param value - value of the setting
* @return same instance of the builder
* @see Client.Builder#serverSetting(String, Collection)
*/
public QuerySettings serverSetting(String name, String value) {
settings.serverSetting(name, value);
return this;
}
/**
* {@see #serverSetting(String, String)} but for multiple values.
*
* @param name - name of the setting without special prefix
* @param values - collection of values
* @return same instance of the builder
*/
public QuerySettings serverSetting(String name, Collection<String> values) {
settings.serverSetting(name, values);
return this;
}
/**
* Sets DB roles for an operation. Roles that were set by {@link Client#setDBRoles(Collection)} will be overridden.
*
* @param dbRoles - list of role to use with an operation
*/
public QuerySettings setDBRoles(Collection<String> dbRoles) {
settings.setDBRoles(dbRoles);
return this;
}
/**
* Gets DB roles for an operation.
*
* @return list of DB roles
*/
public Collection<String> getDBRoles() {
return settings.getDBRoles();
}
/**
* Sets the comment that will be added to the query log record associated with the query.
*
* @param logComment - comment to be added to the log
* @return same instance of the builder
*/
public QuerySettings logComment(String logComment) {
settings.logComment(logComment);
return this;
}
public String getLogComment() {
return settings.getLogComment();
}
/**
* Sets a network operation timeout.
* @param timeout
* @param unit
*/
public void setNetworkTimeout(long timeout, ChronoUnit unit) {
settings.setNetworkTimeout(timeout, unit);
}
/**
* Returns network timeout. Zero value is returned if no timeout is set.
* @return timeout in ms.
*/
public Long getNetworkTimeout() {
return settings.getNetworkTimeout();
}
public static QuerySettings merge(QuerySettings source, QuerySettings override) {
CommonSettings mergedSettings = source.settings.copyAndMerge(override.settings);
return new QuerySettings(mergedSettings);
}
}