Skip to content

Commit 3902d47

Browse files
authored
Merge pull request #1569 from ClickHouse/feature-set-role
Feature: Persist effect of "SET ROLE" queries.
2 parents fdb8205 + 5d48f52 commit 3902d47

File tree

16 files changed

+390
-53
lines changed

16 files changed

+390
-53
lines changed

CHANGELOG.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22

33
### New Features
44
- Added possibility to set client ID in `Referer` HTTP Header (https://github.com/ClickHouse/clickhouse-java/issues/1572)
5+
- [HTTP] Persistence of a role after it is set by `SET ROLE <role>`
56

67
### Bug Fixes
78
- Change RowBinaryWithDefaults settings. Output is changed from true to false

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ public class ClickHouseException extends Exception {
1818
*/
1919
private static final long serialVersionUID = -2417038200885554382L;
2020

21+
public static final int ERROR_UNKNOWN_SETTING = 115;
2122
public static final int ERROR_ABORTED = 236;
2223
public static final int ERROR_CANCELLED = 394;
2324
public static final int ERROR_NETWORK = 210;

clickhouse-client/src/test/resources/containers/clickhouse-server/users.d/users.xml

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,5 +67,14 @@
6767
<common_name>me</common_name>
6868
</ssl_certificates>
6969
</me>
70+
<access_dba>
71+
<profile>default</profile>
72+
<networks incl="networks" replace="replace">
73+
<ip>::/0</ip>
74+
</networks>
75+
<password>123</password>
76+
<quota>default</quota>
77+
<access_management>1</access_management>
78+
</access_dba>
7079
</users>
7180
</clickhouse>

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ApacheHttpConnectionImpl.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@
6060
import java.net.Socket;
6161
import java.net.StandardSocketOptions;
6262
import java.nio.charset.StandardCharsets;
63+
import java.util.Collections;
6364
import java.util.List;
6465
import java.util.Map;
6566
import java.util.TimeZone;
@@ -76,7 +77,7 @@ public class ApacheHttpConnectionImpl extends ClickHouseHttpConnection {
7677

7778
protected ApacheHttpConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor)
7879
throws IOException {
79-
super(server, request);
80+
super(server, request, Collections.emptyMap());
8081

8182
client = newConnection(config);
8283
}

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java

Lines changed: 51 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -1,30 +1,36 @@
11
package com.clickhouse.client.http;
22

3-
import java.io.IOException;
4-
import java.io.UncheckedIOException;
5-
import java.net.InetAddress;
6-
import java.net.NetworkInterface;
7-
import java.net.SocketException;
8-
import java.nio.charset.StandardCharsets;
9-
import java.util.*;
10-
import java.util.concurrent.CompletionException;
11-
123
import com.clickhouse.client.AbstractClient;
134
import com.clickhouse.client.ClickHouseConfig;
145
import com.clickhouse.client.ClickHouseException;
156
import com.clickhouse.client.ClickHouseNode;
167
import com.clickhouse.client.ClickHouseProtocol;
178
import com.clickhouse.client.ClickHouseRequest;
189
import com.clickhouse.client.ClickHouseResponse;
19-
import com.clickhouse.client.ClickHouseTransaction;
2010
import com.clickhouse.client.ClickHouseStreamResponse;
21-
import com.clickhouse.client.config.ClickHouseClientOption;
11+
import com.clickhouse.client.ClickHouseTransaction;
2212
import com.clickhouse.client.http.config.ClickHouseHttpOption;
2313
import com.clickhouse.config.ClickHouseOption;
2414
import com.clickhouse.data.ClickHouseChecker;
2515
import com.clickhouse.logging.Logger;
2616
import com.clickhouse.logging.LoggerFactory;
2717

18+
import java.io.IOException;
19+
import java.io.Serializable;
20+
import java.io.UncheckedIOException;
21+
import java.net.InetAddress;
22+
import java.net.NetworkInterface;
23+
import java.net.SocketException;
24+
import java.nio.charset.StandardCharsets;
25+
import java.util.Collection;
26+
import java.util.Collections;
27+
import java.util.Enumeration;
28+
import java.util.List;
29+
import java.util.Map;
30+
import java.util.Set;
31+
import java.util.concurrent.CompletionException;
32+
import java.util.concurrent.ConcurrentSkipListSet;
33+
2834
public class ClickHouseHttpClient extends AbstractClient<ClickHouseHttpConnection> {
2935
private static final Logger log = LoggerFactory.getLogger(ClickHouseHttpClient.class);
3036

@@ -81,6 +87,8 @@ public ClickHouseHttpClient() {
8187
}
8288
}
8389

90+
private ConcurrentSkipListSet<String> roles = new ConcurrentSkipListSet<>();
91+
8492
@Override
8593
protected boolean checkConnection(ClickHouseHttpConnection connection, ClickHouseNode requestServer,
8694
ClickHouseNode currentServer, ClickHouseRequest<?> request) {
@@ -106,7 +114,8 @@ protected ClickHouseHttpConnection newConnection(ClickHouseHttpConnection connec
106114
}
107115

108116
try {
109-
return ClickHouseHttpConnectionFactory.createConnection(server, request, getExecutor());
117+
118+
return ClickHouseHttpConnectionFactory.createConnection(server, request, getExecutor(), buildAdditionalReqParams(request));
110119
} catch (IOException e) {
111120
throw new CompletionException(e);
112121
}
@@ -138,6 +147,18 @@ protected String buildQueryParams(Map<String, String> params) {
138147
return builder.toString();
139148
}
140149

150+
private Map<String, Serializable> buildAdditionalReqParams(ClickHouseRequest<?> sealedRequest) {
151+
ClickHouseConfig config = sealedRequest.getConfig();
152+
if (config.getBoolOption(ClickHouseHttpOption.REMEMBER_LAST_SET_ROLES)) {
153+
if (sealedRequest.hasSetting("_set_roles_stmt")) {
154+
return Collections.singletonMap("_roles", sealedRequest.getSettings().get("_set_roles_stmt"));
155+
} else if (!roles.isEmpty()) {
156+
return Collections.singletonMap("_roles", roles);
157+
}
158+
}
159+
return Collections.emptyMap();
160+
}
161+
141162
@Override
142163
protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws ClickHouseException, IOException {
143164
ClickHouseHttpConnection conn = getConnection(sealedRequest);
@@ -166,18 +187,30 @@ protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws Cli
166187
}
167188
}
168189
: null;
190+
169191
if (conn.isReusable()) {
192+
Map<String, Serializable> additionalParams = buildAdditionalReqParams(sealedRequest);
193+
170194
ClickHouseNode server = sealedRequest.getServer();
171195
httpResponse = conn.post(config, sql, sealedRequest.getInputStream().orElse(null),
172196
sealedRequest.getExternalTables(), sealedRequest.getOutputStream().orElse(null),
173-
ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest),
197+
ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest, additionalParams),
174198
ClickHouseHttpConnection.createDefaultHeaders(config, server, conn.getUserAgent(), getReferer(config)),
175199
postAction);
176200
} else {
177201
httpResponse = conn.post(config, sql, sealedRequest.getInputStream().orElse(null),
178202
sealedRequest.getExternalTables(), sealedRequest.getOutputStream().orElse(null), null, null,
179203
postAction);
180204
}
205+
206+
if (config.getBoolOption(ClickHouseHttpOption.REMEMBER_LAST_SET_ROLES)) {
207+
// At this point only successful responses are expected
208+
if (sealedRequest.hasSetting("_set_roles_stmt")) {
209+
rememberRoles((Set<String>) sealedRequest.getSettings().get("_set_roles_stmt"));
210+
}
211+
}
212+
213+
181214
return ClickHouseStreamResponse.of(httpResponse.getConfig(sealedRequest), httpResponse.getInputStream(),
182215
sealedRequest.getSettings(), null, httpResponse.summary);
183216
}
@@ -186,4 +219,9 @@ protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws Cli
186219
public final Class<? extends ClickHouseOption> getOptionClass() {
187220
return ClickHouseHttpOption.class;
188221
}
222+
223+
private void rememberRoles(Set<String> requestedRoles) {
224+
roles.clear();
225+
roles.addAll(requestedRoles);
226+
}
189227
}

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java

Lines changed: 32 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -4,11 +4,20 @@
44
import java.io.OutputStream;
55
import java.io.Serializable;
66
import java.io.UnsupportedEncodingException;
7-
import java.net.*;
7+
import java.net.InetSocketAddress;
8+
import java.net.Proxy;
9+
import java.net.URLEncoder;
810
import java.nio.charset.Charset;
911
import java.nio.charset.StandardCharsets;
10-
import java.util.*;
12+
import java.util.Collections;
13+
import java.util.HashSet;
14+
import java.util.LinkedHashMap;
15+
import java.util.List;
16+
import java.util.Locale;
17+
import java.util.Map;
18+
import java.util.Optional;
1119
import java.util.Map.Entry;
20+
import java.util.Set;
1221

1322
import com.clickhouse.client.ClickHouseClient;
1423
import com.clickhouse.client.ClickHouseConfig;
@@ -70,11 +79,15 @@ static String urlEncode(String str, Charset charset) {
7079
}
7180
}
7281

73-
static String buildQueryParams(ClickHouseRequest<?> request) {
82+
static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
7483
if (request == null) {
7584
return "";
7685
}
7786

87+
if (additionalParams == null) {
88+
additionalParams = Collections.emptyMap();
89+
}
90+
7891
ClickHouseConfig config = request.getConfig();
7992
StringBuilder builder = new StringBuilder();
8093

@@ -127,6 +140,13 @@ static String buildQueryParams(ClickHouseRequest<?> request) {
127140
appendQueryParameter(builder, settingKey, "0");
128141
}
129142

143+
// Handle additional parameters
144+
if (additionalParams.containsKey("_roles")) {
145+
Serializable value = additionalParams.get("_roles");
146+
Set<String> roles = !(value instanceof Set) ? Collections.emptySet() : (Set<String>) value;
147+
roles.forEach(role -> appendQueryParameter(builder, "role", role));
148+
}
149+
130150
Optional<String> optionalValue = request.getSessionId();
131151
if (optionalValue.isPresent()) {
132152
appendQueryParameter(builder, ClickHouseClientOption.SESSION_ID.getKey(), optionalValue.get());
@@ -147,6 +167,10 @@ static String buildQueryParams(ClickHouseRequest<?> request) {
147167
}
148168

149169
for (Entry<String, Serializable> entry : settings.entrySet()) {
170+
// Skip internal settings
171+
if (entry.getKey().equalsIgnoreCase("_set_roles_stmt")) {
172+
continue;
173+
}
150174
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
151175
}
152176

@@ -156,7 +180,7 @@ static String buildQueryParams(ClickHouseRequest<?> request) {
156180
return builder.toString();
157181
}
158182

159-
static String buildUrl(String baseUrl, ClickHouseRequest<?> request) {
183+
static String buildUrl(String baseUrl, ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
160184
StringBuilder builder = new StringBuilder().append(baseUrl);
161185
// TODO: Using default until we will remove
162186
String context = "/";
@@ -173,7 +197,7 @@ static String buildUrl(String baseUrl, ClickHouseRequest<?> request) {
173197
}
174198
}
175199

176-
String query = buildQueryParams(request);
200+
String query = buildQueryParams(request, additionalParams);
177201
if (!query.isEmpty()) {
178202
builder.append('?').append(query);
179203
}
@@ -350,7 +374,8 @@ protected static void postData(ClickHouseConfig config, byte[] boundary, String
350374
protected final Map<String, String> defaultHeaders;
351375
protected final String url;
352376

353-
protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request) {
377+
protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request,
378+
Map<String, Serializable> additionalParams) {
354379
if (server == null || request == null) {
355380
throw new IllegalArgumentException("Non-null server and request are required");
356381
}
@@ -361,7 +386,7 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> r
361386
ClickHouseConfig c = request.getConfig();
362387
this.config = c;
363388
this.defaultHeaders = Collections.unmodifiableMap(createDefaultHeaders(c, server, getUserAgent(), ClickHouseHttpClient.getReferer(config)));
364-
this.url = buildUrl(server.getBaseUri(), request);
389+
this.url = buildUrl(server.getBaseUri(), request, additionalParams);
365390
log.debug("url [%s]", this.url);
366391
}
367392

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,9 @@
11
package com.clickhouse.client.http;
22

33
import java.io.IOException;
4+
import java.io.Serializable;
5+
import java.util.Collections;
6+
import java.util.Map;
47
import java.util.concurrent.ExecutorService;
58

69
import com.clickhouse.client.ClickHouseNode;
@@ -14,7 +17,14 @@ public final class ClickHouseHttpConnectionFactory {
1417
private static final Logger log = LoggerFactory.getLogger(ClickHouseHttpConnectionFactory.class);
1518

1619
public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest<?> request,
17-
ExecutorService executor) throws IOException {
20+
ExecutorService executor) throws IOException
21+
{
22+
return createConnection(server, request, executor, Collections.emptyMap());
23+
}
24+
25+
public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest<?> request,
26+
ExecutorService executor, Map<String,
27+
Serializable> additionalRequestParams) throws IOException {
1828
HttpConnectionProvider provider = request.getConfig().getOption(ClickHouseHttpOption.CONNECTION_PROVIDER,
1929
HttpConnectionProvider.class);
2030
if (provider == HttpConnectionProvider.APACHE_HTTP_CLIENT) {
@@ -27,7 +37,7 @@ public static ClickHouseHttpConnection createConnection(ClickHouseNode server, C
2737
log.warn("HTTP_CLIENT is only supported in JDK 11 or above, fall back to HTTP_URL_CONNECTION");
2838
}
2939

30-
return new HttpUrlConnectionImpl(server, request, executor);
40+
return new HttpUrlConnectionImpl(server, request, executor, additionalRequestParams);
3141
}
3242

3343
private ClickHouseHttpConnectionFactory() {

clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java

Lines changed: 8 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,18 @@
1717
import com.clickhouse.logging.Logger;
1818
import com.clickhouse.logging.LoggerFactory;
1919

20+
import javax.net.ssl.HostnameVerifier;
21+
import javax.net.ssl.HttpsURLConnection;
22+
import javax.net.ssl.SSLContext;
2023
import java.io.BufferedReader;
2124
import java.io.ByteArrayInputStream;
2225
import java.io.ByteArrayOutputStream;
2326
import java.io.IOException;
2427
import java.io.InputStream;
2528
import java.io.InputStreamReader;
2629
import java.io.OutputStream;
30+
import java.io.Serializable;
2731
import java.io.UncheckedIOException;
28-
2932
import java.net.ConnectException;
3033
import java.net.HttpURLConnection;
3134
import java.net.Proxy;
@@ -35,15 +38,11 @@
3538
import java.util.HashSet;
3639
import java.util.List;
3740
import java.util.Map;
41+
import java.util.Map.Entry;
3842
import java.util.Set;
3943
import java.util.TimeZone;
40-
import java.util.Map.Entry;
4144
import java.util.concurrent.ExecutorService;
4245

43-
import javax.net.ssl.HostnameVerifier;
44-
import javax.net.ssl.HttpsURLConnection;
45-
import javax.net.ssl.SSLContext;
46-
4746
public class HttpUrlConnectionImpl extends ClickHouseHttpConnection {
4847
private static final Logger log = LoggerFactory.getLogger(HttpUrlConnectionImpl.class);
4948

@@ -205,9 +204,9 @@ private void checkResponse(HttpURLConnection conn) throws IOException {
205204
}
206205
}
207206

208-
protected HttpUrlConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor)
209-
throws IOException {
210-
super(server, request);
207+
protected HttpUrlConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor,
208+
Map<String, Serializable> additionalParams) throws IOException {
209+
super(server, request, additionalParams);
211210

212211
conn = newConnection(url, true);
213212
}

clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,14 +51,21 @@ public enum ClickHouseHttpOption implements ClickHouseOption {
5151
* 3. HOST_NAME - host name is used
5252
*/
5353
SEND_HTTP_CLIENT_ID("send_http_client_id", "", "Indicates whether http client would send its identification through Referer header to server. " +
54-
"Valid values: empty string - nothing is sent. IP_ADDRESS - client's IP address is used. HOST_NAME - host name is used.");
54+
"Valid values: empty string - nothing is sent. IP_ADDRESS - client's IP address is used. HOST_NAME - host name is used."),
5555

5656
// SEND_PROGRESS("send_progress_in_http_headers", false,
5757
// "Enables or disables X-ClickHouse-Progress HTTP response headers in
5858
// clickhouse-server responses."),
5959
// SEND_PROGRESS_INTERVAL("http_headers_progress_interval_ms", 3000, ""),
6060
// WAIT_END_OF_QUERY("wait_end_of_query", false, ""),
6161

62+
/**
63+
* Whether to remember last set role and send them in every next requests as query parameters.
64+
* Only one role can be set at a time.
65+
*/
66+
REMEMBER_LAST_SET_ROLES("remember_last_set_roles", false,
67+
"Whether to remember last set role and send them in every next requests as query parameters.");
68+
6269
private final String key;
6370
private final Serializable defaultValue;
6471
private final Class<? extends Serializable> clazz;

0 commit comments

Comments
 (0)