Skip to content

Commit

Permalink
Merge pull request #1569 from ClickHouse/feature-set-role
Browse files Browse the repository at this point in the history
Feature: Persist effect of "SET ROLE" queries.
  • Loading branch information
chernser authored Apr 24, 2024
2 parents fdb8205 + 5d48f52 commit 3902d47
Show file tree
Hide file tree
Showing 16 changed files with 390 additions and 53 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@

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

### Bug Fixes
- Change RowBinaryWithDefaults settings. Output is changed from true to false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ public class ClickHouseException extends Exception {
*/
private static final long serialVersionUID = -2417038200885554382L;

public static final int ERROR_UNKNOWN_SETTING = 115;
public static final int ERROR_ABORTED = 236;
public static final int ERROR_CANCELLED = 394;
public static final int ERROR_NETWORK = 210;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,5 +67,14 @@
<common_name>me</common_name>
</ssl_certificates>
</me>
<access_dba>
<profile>default</profile>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<password>123</password>
<quota>default</quota>
<access_management>1</access_management>
</access_dba>
</users>
</clickhouse>
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import java.net.Socket;
import java.net.StandardSocketOptions;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TimeZone;
Expand All @@ -76,7 +77,7 @@ public class ApacheHttpConnectionImpl extends ClickHouseHttpConnection {

protected ApacheHttpConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor)
throws IOException {
super(server, request);
super(server, request, Collections.emptyMap());

client = newConnection(config);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,30 +1,36 @@
package com.clickhouse.client.http;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.nio.charset.StandardCharsets;
import java.util.*;
import java.util.concurrent.CompletionException;

import com.clickhouse.client.AbstractClient;
import com.clickhouse.client.ClickHouseConfig;
import com.clickhouse.client.ClickHouseException;
import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseProtocol;
import com.clickhouse.client.ClickHouseRequest;
import com.clickhouse.client.ClickHouseResponse;
import com.clickhouse.client.ClickHouseTransaction;
import com.clickhouse.client.ClickHouseStreamResponse;
import com.clickhouse.client.config.ClickHouseClientOption;
import com.clickhouse.client.ClickHouseTransaction;
import com.clickhouse.client.http.config.ClickHouseHttpOption;
import com.clickhouse.config.ClickHouseOption;
import com.clickhouse.data.ClickHouseChecker;
import com.clickhouse.logging.Logger;
import com.clickhouse.logging.LoggerFactory;

import java.io.IOException;
import java.io.Serializable;
import java.io.UncheckedIOException;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentSkipListSet;

public class ClickHouseHttpClient extends AbstractClient<ClickHouseHttpConnection> {
private static final Logger log = LoggerFactory.getLogger(ClickHouseHttpClient.class);

Expand Down Expand Up @@ -81,6 +87,8 @@ public ClickHouseHttpClient() {
}
}

private ConcurrentSkipListSet<String> roles = new ConcurrentSkipListSet<>();

@Override
protected boolean checkConnection(ClickHouseHttpConnection connection, ClickHouseNode requestServer,
ClickHouseNode currentServer, ClickHouseRequest<?> request) {
Expand All @@ -106,7 +114,8 @@ protected ClickHouseHttpConnection newConnection(ClickHouseHttpConnection connec
}

try {
return ClickHouseHttpConnectionFactory.createConnection(server, request, getExecutor());

return ClickHouseHttpConnectionFactory.createConnection(server, request, getExecutor(), buildAdditionalReqParams(request));
} catch (IOException e) {
throw new CompletionException(e);
}
Expand Down Expand Up @@ -138,6 +147,18 @@ protected String buildQueryParams(Map<String, String> params) {
return builder.toString();
}

private Map<String, Serializable> buildAdditionalReqParams(ClickHouseRequest<?> sealedRequest) {
ClickHouseConfig config = sealedRequest.getConfig();
if (config.getBoolOption(ClickHouseHttpOption.REMEMBER_LAST_SET_ROLES)) {
if (sealedRequest.hasSetting("_set_roles_stmt")) {
return Collections.singletonMap("_roles", sealedRequest.getSettings().get("_set_roles_stmt"));
} else if (!roles.isEmpty()) {
return Collections.singletonMap("_roles", roles);
}
}
return Collections.emptyMap();
}

@Override
protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws ClickHouseException, IOException {
ClickHouseHttpConnection conn = getConnection(sealedRequest);
Expand Down Expand Up @@ -166,18 +187,30 @@ protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws Cli
}
}
: null;

if (conn.isReusable()) {
Map<String, Serializable> additionalParams = buildAdditionalReqParams(sealedRequest);

ClickHouseNode server = sealedRequest.getServer();
httpResponse = conn.post(config, sql, sealedRequest.getInputStream().orElse(null),
sealedRequest.getExternalTables(), sealedRequest.getOutputStream().orElse(null),
ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest),
ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest, additionalParams),
ClickHouseHttpConnection.createDefaultHeaders(config, server, conn.getUserAgent(), getReferer(config)),
postAction);
} else {
httpResponse = conn.post(config, sql, sealedRequest.getInputStream().orElse(null),
sealedRequest.getExternalTables(), sealedRequest.getOutputStream().orElse(null), null, null,
postAction);
}

if (config.getBoolOption(ClickHouseHttpOption.REMEMBER_LAST_SET_ROLES)) {
// At this point only successful responses are expected
if (sealedRequest.hasSetting("_set_roles_stmt")) {
rememberRoles((Set<String>) sealedRequest.getSettings().get("_set_roles_stmt"));
}
}


return ClickHouseStreamResponse.of(httpResponse.getConfig(sealedRequest), httpResponse.getInputStream(),
sealedRequest.getSettings(), null, httpResponse.summary);
}
Expand All @@ -186,4 +219,9 @@ protected ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) throws Cli
public final Class<? extends ClickHouseOption> getOptionClass() {
return ClickHouseHttpOption.class;
}

private void rememberRoles(Set<String> requestedRoles) {
roles.clear();
roles.addAll(requestedRoles);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -4,11 +4,20 @@
import java.io.OutputStream;
import java.io.Serializable;
import java.io.UnsupportedEncodingException;
import java.net.*;
import java.net.InetSocketAddress;
import java.net.Proxy;
import java.net.URLEncoder;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.*;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Map.Entry;
import java.util.Set;

import com.clickhouse.client.ClickHouseClient;
import com.clickhouse.client.ClickHouseConfig;
Expand Down Expand Up @@ -70,11 +79,15 @@ static String urlEncode(String str, Charset charset) {
}
}

static String buildQueryParams(ClickHouseRequest<?> request) {
static String buildQueryParams(ClickHouseRequest<?> request, Map<String, Serializable> additionalParams) {
if (request == null) {
return "";
}

if (additionalParams == null) {
additionalParams = Collections.emptyMap();
}

ClickHouseConfig config = request.getConfig();
StringBuilder builder = new StringBuilder();

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

// Handle additional parameters
if (additionalParams.containsKey("_roles")) {
Serializable value = additionalParams.get("_roles");
Set<String> roles = !(value instanceof Set) ? Collections.emptySet() : (Set<String>) value;
roles.forEach(role -> appendQueryParameter(builder, "role", role));
}

Optional<String> optionalValue = request.getSessionId();
if (optionalValue.isPresent()) {
appendQueryParameter(builder, ClickHouseClientOption.SESSION_ID.getKey(), optionalValue.get());
Expand All @@ -147,6 +167,10 @@ static String buildQueryParams(ClickHouseRequest<?> request) {
}

for (Entry<String, Serializable> entry : settings.entrySet()) {
// Skip internal settings
if (entry.getKey().equalsIgnoreCase("_set_roles_stmt")) {
continue;
}
appendQueryParameter(builder, entry.getKey(), String.valueOf(entry.getValue()));
}

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

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

String query = buildQueryParams(request);
String query = buildQueryParams(request, additionalParams);
if (!query.isEmpty()) {
builder.append('?').append(query);
}
Expand Down Expand Up @@ -350,7 +374,8 @@ protected static void postData(ClickHouseConfig config, byte[] boundary, String
protected final Map<String, String> defaultHeaders;
protected final String url;

protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request) {
protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> request,
Map<String, Serializable> additionalParams) {
if (server == null || request == null) {
throw new IllegalArgumentException("Non-null server and request are required");
}
Expand All @@ -361,7 +386,7 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> r
ClickHouseConfig c = request.getConfig();
this.config = c;
this.defaultHeaders = Collections.unmodifiableMap(createDefaultHeaders(c, server, getUserAgent(), ClickHouseHttpClient.getReferer(config)));
this.url = buildUrl(server.getBaseUri(), request);
this.url = buildUrl(server.getBaseUri(), request, additionalParams);
log.debug("url [%s]", this.url);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,9 @@
package com.clickhouse.client.http;

import java.io.IOException;
import java.io.Serializable;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ExecutorService;

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

public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest<?> request,
ExecutorService executor) throws IOException {
ExecutorService executor) throws IOException
{
return createConnection(server, request, executor, Collections.emptyMap());
}

public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest<?> request,
ExecutorService executor, Map<String,
Serializable> additionalRequestParams) throws IOException {
HttpConnectionProvider provider = request.getConfig().getOption(ClickHouseHttpOption.CONNECTION_PROVIDER,
HttpConnectionProvider.class);
if (provider == HttpConnectionProvider.APACHE_HTTP_CLIENT) {
Expand All @@ -27,7 +37,7 @@ public static ClickHouseHttpConnection createConnection(ClickHouseNode server, C
log.warn("HTTP_CLIENT is only supported in JDK 11 or above, fall back to HTTP_URL_CONNECTION");
}

return new HttpUrlConnectionImpl(server, request, executor);
return new HttpUrlConnectionImpl(server, request, executor, additionalRequestParams);
}

private ClickHouseHttpConnectionFactory() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,18 @@
import com.clickhouse.logging.Logger;
import com.clickhouse.logging.LoggerFactory;

import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLContext;
import java.io.BufferedReader;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.Serializable;
import java.io.UncheckedIOException;

import java.net.ConnectException;
import java.net.HttpURLConnection;
import java.net.Proxy;
Expand All @@ -35,15 +38,11 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.TimeZone;
import java.util.Map.Entry;
import java.util.concurrent.ExecutorService;

import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLContext;

public class HttpUrlConnectionImpl extends ClickHouseHttpConnection {
private static final Logger log = LoggerFactory.getLogger(HttpUrlConnectionImpl.class);

Expand Down Expand Up @@ -205,9 +204,9 @@ private void checkResponse(HttpURLConnection conn) throws IOException {
}
}

protected HttpUrlConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor)
throws IOException {
super(server, request);
protected HttpUrlConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> request, ExecutorService executor,
Map<String, Serializable> additionalParams) throws IOException {
super(server, request, additionalParams);

conn = newConnection(url, true);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,14 +51,21 @@ public enum ClickHouseHttpOption implements ClickHouseOption {
* 3. HOST_NAME - host name is used
*/
SEND_HTTP_CLIENT_ID("send_http_client_id", "", "Indicates whether http client would send its identification through Referer header to server. " +
"Valid values: empty string - nothing is sent. IP_ADDRESS - client's IP address is used. HOST_NAME - host name is used.");
"Valid values: empty string - nothing is sent. IP_ADDRESS - client's IP address is used. HOST_NAME - host name is used."),

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

/**
* Whether to remember last set role and send them in every next requests as query parameters.
* Only one role can be set at a time.
*/
REMEMBER_LAST_SET_ROLES("remember_last_set_roles", false,
"Whether to remember last set role and send them in every next requests as query parameters.");

private final String key;
private final Serializable defaultValue;
private final Class<? extends Serializable> clazz;
Expand Down
Loading

0 comments on commit 3902d47

Please sign in to comment.