Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
73 changes: 42 additions & 31 deletions client-v2/src/main/java/com/clickhouse/client/api/Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.clickhouse.client.api.insert.InsertResponse;
import com.clickhouse.client.api.insert.InsertSettings;
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.CredentialsManager;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
import com.clickhouse.client.api.internal.MapUtils;
import com.clickhouse.client.api.internal.TableSchemaParser;
Expand Down Expand Up @@ -141,11 +142,13 @@
private final int retries;
private LZ4Factory lz4Factory = null;
private final Supplier<String> queryIdGenerator;
private final CredentialsManager credentialsManager;

private Client(Collection<Endpoint> endpoints, Map<String,String> configuration,
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy,
Object metricsRegistry, Supplier<String> queryIdGenerator) {
this.configuration = ClientConfigProperties.parseConfigMap(configuration);
this.configuration = new ConcurrentHashMap<>(ClientConfigProperties.parseConfigMap(configuration));
this.credentialsManager = new CredentialsManager(this.configuration);
this.readOnlyConfig = Collections.unmodifiableMap(configuration);
this.metricsRegistry = metricsRegistry;
this.queryIdGenerator = queryIdGenerator;
Expand Down Expand Up @@ -364,8 +367,11 @@
if (key.equals(ClientConfigProperties.PRODUCT_NAME.getKey())) {
setClientName(value);
}
if (key.equals(ClientConfigProperties.ACCESS_TOKEN.getKey())) {
setAccessToken(value);
}
if (key.equals(ClientConfigProperties.BEARERTOKEN_AUTH.getKey())) {
useBearerTokenAuth(value);
setAccessToken(value);
}
return this;
}
Expand Down Expand Up @@ -393,13 +399,17 @@
}

/**
* Access token for authentication with server. Required for all operations.
* Preferred way to configure token-based authentication.
* Same access token will be used for all endpoints.
* Internally it is sent as an HTTP Bearer token.
*
* @param accessToken - plain text access token
*/
@SuppressWarnings("deprecation")
public Builder setAccessToken(String accessToken) {
this.configuration.put(ClientConfigProperties.ACCESS_TOKEN.getKey(), accessToken);
this.configuration.remove(ClientConfigProperties.BEARERTOKEN_AUTH.getKey());
this.httpHeader(HttpHeaders.AUTHORIZATION, "Bearer " + accessToken);
return this;
}

Expand Down Expand Up @@ -1007,16 +1017,16 @@
}

/**
* Specifies whether to use Bearer Authentication and what token to use.
* The token will be sent as is, so it should be encoded before passing to this method.
* Legacy HTTP-specific alias for {@link Builder#setAccessToken(String)}.
* Prefer using {@link Builder#setAccessToken(String)}.
*
* @param bearerToken - token to use
* @return same instance of the builder
*/
@Deprecated
public Builder useBearerTokenAuth(String bearerToken) {

Check warning on line 1027 in client-v2/src/main/java/com/clickhouse/client/api/Client.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Add the missing @deprecated Javadoc tag.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomToMKTo52DUqrNS&open=AZ1AomToMKTo52DUqrNS&pullRequest=2812

Check warning on line 1027 in client-v2/src/main/java/com/clickhouse/client/api/Client.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Do not forget to remove this deprecated code someday.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomToMKTo52DUqrNR&open=AZ1AomToMKTo52DUqrNR&pullRequest=2812
// Most JWT libraries (https://jwt.io/libraries?language=Java) compact tokens in proper way
this.httpHeader(HttpHeaders.AUTHORIZATION, "Bearer " + bearerToken);
return this;
return setAccessToken(bearerToken);
}

/**
Expand Down Expand Up @@ -1099,28 +1109,10 @@
if (this.endpoints.isEmpty()) {
throw new IllegalArgumentException("At least one endpoint is required");
}
// check if username and password are empty. so can not initiate client?
boolean useSslAuth = MapUtils.getFlag(this.configuration, ClientConfigProperties.SSL_AUTH.getKey());
boolean hasAccessToken = this.configuration.containsKey(ClientConfigProperties.ACCESS_TOKEN.getKey());
boolean hasUser = this.configuration.containsKey(ClientConfigProperties.USER.getKey());
boolean hasPassword = this.configuration.containsKey(ClientConfigProperties.PASSWORD.getKey());
boolean customHttpHeaders = this.configuration.containsKey(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION));

if (!(useSslAuth || hasAccessToken || hasUser || hasPassword || customHttpHeaders)) {
throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required");
}

if (useSslAuth && (hasAccessToken || hasPassword)) {
throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client.");
}

if (useSslAuth && !this.configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
throw new IllegalArgumentException("SSL authentication requires a client certificate");
}

if (this.configuration.containsKey(ClientConfigProperties.SSL_TRUST_STORE.getKey()) &&
this.configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
throw new IllegalArgumentException("Trust store and certificates cannot be used together");
ClientMisconfigurationException authConfigException = CredentialsManager.validateAuthConfig(configuration);
if (authConfigException != null) {
throw authConfigException;
}

// Check timezone settings
Expand Down Expand Up @@ -2153,8 +2145,28 @@
return unmodifiableDbRolesView;
}

public void setCredentials(String username, String password) {
this.credentialsManager.setCredentials(username, password);
}

/**
* Preferred runtime API to update token-based authentication.
* Internally it refreshes the HTTP Bearer token used by requests.
*
* @param accessToken - plain text access token
*/
public void setAccessToken(String accessToken) {
this.credentialsManager.setAccessToken(accessToken);
}

/**
* Legacy HTTP-specific alias for {@link #setAccessToken(String)}.
* Prefer using {@link #setAccessToken(String)}.
*
* @param bearer - token to use
*/
public void updateBearerToken(String bearer) {
this.configuration.put(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION), "Bearer " + bearer);
setAccessToken(bearer);
}

private Endpoint getNextAliveNode() {
Expand All @@ -2170,8 +2182,7 @@
* @return request settings - merged client and operation settings
*/
private Map<String, Object> buildRequestSettings(Map<String, Object> opSettings) {
Map<String, Object> requestSettings = new HashMap<>();
requestSettings.putAll(configuration);
Map<String, Object> requestSettings = credentialsManager.snapshot();
requestSettings.putAll(opSettings);
return requestSettings;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,10 @@ public enum ClientConfigProperties {

CLIENT_NETWORK_BUFFER_SIZE("client_network_buffer_size", Integer.class, "300000"),

/**
* Preferred client setting for token-based authentication like JWT and Oauth.
* For Http it is translated to Authorization Bearer header.
*/
ACCESS_TOKEN("access_token", String.class),

SSL_AUTH("ssl_authentication", Boolean.class, "false"),
Expand Down Expand Up @@ -157,6 +161,10 @@ public Object parseValue(String value) {
@Deprecated
PRODUCT_NAME("product_name", String.class),

/**
* HTTP-specific alias for {@link ClientConfigProperties#ACCESS_TOKEN}.
* Prefer using {@link ClientConfigProperties#ACCESS_TOKEN}.
*/
BEARERTOKEN_AUTH ("bearer_token", String.class),
/**
* Indicates that data provided for write operation is compressed by application.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
package com.clickhouse.client.api.internal;

import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientMisconfigurationException;
import org.apache.hc.core5.http.HttpHeaders;

import java.util.HashMap;
import java.util.Map;

/**
* Manages mutable authentication-related client settings.
*/
public class CredentialsManager {
private final Map<String, Object> configuration;
private final Object lock = new Object();

public CredentialsManager(Map<String, Object> configuration) {
this.configuration = configuration;
}

public Map<String, Object> snapshot() {
synchronized (lock) {
return new HashMap<>(configuration);
}
}

public void setCredentials(String username, String password) {
synchronized (lock) {
configuration.put(ClientConfigProperties.USER.getKey(), username);
configuration.put(ClientConfigProperties.PASSWORD.getKey(), password);
configuration.put(ClientConfigProperties.SSL_AUTH.getKey(), Boolean.FALSE);
configuration.remove(ClientConfigProperties.ACCESS_TOKEN.getKey());
configuration.remove(ClientConfigProperties.BEARERTOKEN_AUTH.getKey());
configuration.remove(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION));
}
}

public void setAccessToken(String accessToken) {
synchronized (lock) {
configuration.put(ClientConfigProperties.ACCESS_TOKEN.getKey(), accessToken);
configuration.put(ClientConfigProperties.SSL_AUTH.getKey(), Boolean.FALSE);
configuration.remove(ClientConfigProperties.BEARERTOKEN_AUTH.getKey());
configuration.remove(ClientConfigProperties.USER.getKey());
configuration.remove(ClientConfigProperties.PASSWORD.getKey());
configuration.put(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION), "Bearer " + accessToken);
}
}

public static ClientMisconfigurationException validateAuthConfig(Map<String, String> configuration) {
// check if username and password are empty. so can not initiate client?
boolean useSslAuth = MapUtils.getFlag(configuration, ClientConfigProperties.SSL_AUTH.getKey());
boolean hasAccessToken = configuration.containsKey(ClientConfigProperties.ACCESS_TOKEN.getKey());
boolean hasUser = configuration.containsKey(ClientConfigProperties.USER.getKey());
boolean hasPassword = configuration.containsKey(ClientConfigProperties.PASSWORD.getKey());
boolean customHttpHeaders = configuration.containsKey(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION));

if (!(useSslAuth || hasAccessToken || hasUser || hasPassword || customHttpHeaders)) {
return new ClientMisconfigurationException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required");
}

if (useSslAuth && (hasAccessToken || hasPassword)) {
return new ClientMisconfigurationException("Only one of password, access token or SSL authentication can be used per client.");
}

if (useSslAuth && !configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
return new ClientMisconfigurationException("SSL authentication requires a client certificate");
}

if (configuration.containsKey(ClientConfigProperties.SSL_TRUST_STORE.getKey()) &&
configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) {
return new ClientMisconfigurationException("Trust store and certificates cannot be used together");
}

return null;
}
}
34 changes: 34 additions & 0 deletions client-v2/src/test/java/com/clickhouse/client/ClientTests.java
Original file line number Diff line number Diff line change
Expand Up @@ -432,6 +432,40 @@
}
}

@Test(groups = {"integration"})
public void testRuntimeCredentialChange() throws Exception {
if (isCloud()) {
return; // creating users is not expected in cloud tests
}

String user1 = "client_v2_user1_" + RandomStringUtils.random(8, true, true).toLowerCase();

Check warning on line 441 in client-v2/src/test/java/com/clickhouse/client/ClientTests.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Remove this use of "random"; it is deprecated.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomRhMKTo52DUqrNN&open=AZ1AomRhMKTo52DUqrNN&pullRequest=2812
String user2 = "client_v2_user2_" + RandomStringUtils.random(8, true, true).toLowerCase();

Check warning on line 442 in client-v2/src/test/java/com/clickhouse/client/ClientTests.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Remove this use of "random"; it is deprecated.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomRhMKTo52DUqrNO&open=AZ1AomRhMKTo52DUqrNO&pullRequest=2812
String password1 = "^1A" + RandomStringUtils.random(12, true, true) + "3b$";

Check warning on line 443 in client-v2/src/test/java/com/clickhouse/client/ClientTests.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Remove this use of "random"; it is deprecated.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomRhMKTo52DUqrNP&open=AZ1AomRhMKTo52DUqrNP&pullRequest=2812
String password2 = "^1A" + RandomStringUtils.random(12, true, true) + "3B$";

Check warning on line 444 in client-v2/src/test/java/com/clickhouse/client/ClientTests.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Remove this use of "random"; it is deprecated.

See more on https://sonarcloud.io/project/issues?id=ClickHouse_clickhouse-java&issues=AZ1AomRhMKTo52DUqrNQ&open=AZ1AomRhMKTo52DUqrNQ&pullRequest=2812

try (Client adminClient = newClient().build()) {
try {
adminClient.execute("DROP USER IF EXISTS " + user1).get().close();
adminClient.execute("DROP USER IF EXISTS " + user2).get().close();
adminClient.execute("CREATE USER " + user1 + " IDENTIFIED BY '" + password1 + "'").get().close();
adminClient.execute("CREATE USER " + user2 + " IDENTIFIED BY '" + password2 + "'").get().close();

try (Client userClient = newClient().setUsername(user1).setPassword(password1).build()) {
List<GenericRecord> firstResponse = userClient.queryAll("SELECT currentUser() AS user");
Assert.assertEquals(firstResponse.get(0).getString("user"), user1);

userClient.setCredentials(user2, password2);

List<GenericRecord> secondResponse = userClient.queryAll("SELECT currentUser() AS user");
Assert.assertEquals(secondResponse.get(0).getString("user"), user2);
}
} finally {
adminClient.execute("DROP USER IF EXISTS " + user1).get().close();
adminClient.execute("DROP USER IF EXISTS " + user2).get().close();
}
}
}


@Test(groups = {"integration"})
public void testLogComment() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -944,7 +944,7 @@ public void testBearerTokenAuth() throws Exception {
.map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8)))
.reduce((s1, s2) -> s1 + "." + s2).get();
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.useBearerTokenAuth(jwtToken1)
.setAccessToken(jwtToken1)
.compressServerResponse(false)
.build()) {

Expand Down Expand Up @@ -974,7 +974,7 @@ public void testBearerTokenAuth() throws Exception {
.build());

try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.useBearerTokenAuth(jwtToken1)
.setAccessToken(jwtToken1)
.compressServerResponse(false)
.build()) {

Expand All @@ -994,7 +994,7 @@ public void testBearerTokenAuth() throws Exception {

.build());

client.updateBearerToken(jwtToken2);
client.setAccessToken(jwtToken2);

client.execute("SELECT 1").get();
}
Expand All @@ -1003,6 +1003,59 @@ public void testBearerTokenAuth() throws Exception {
}
}

@Test(groups = { "integration" })
public void testSetCredentialsAfterClientCreation() throws Exception {
if (isCloud()) {
return; // mocked server
}

WireMockServer mockServer = new WireMockServer(WireMockConfiguration
.options().port(9090).notifier(new ConsoleNotifier(false)));
mockServer.start();

try {
String user1 = "default";
String password1 = "wrong-password";
String user2 = "runtime-user";
String password2 = "runtime-password";
String basicAuth2 = "Basic " + Base64.getEncoder().encodeToString(
(user2 + ":" + password2).getBytes(StandardCharsets.UTF_8));

mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.withHeader(HttpHeaders.AUTHORIZATION, WireMock.equalTo("Basic " + Base64.getEncoder().encodeToString(
(user1 + ":" + password1).getBytes(StandardCharsets.UTF_8))))
.willReturn(WireMock.aResponse()
.withStatus(HttpStatus.SC_UNAUTHORIZED))
.build());

try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.setUsername(user1)
.setPassword(password1)
.compressServerResponse(false)
.build()) {
try {
client.execute("SELECT 1").get();
fail("Exception expected");
} catch (ServerException e) {
Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED);
}

mockServer.resetAll();
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.withHeader(HttpHeaders.AUTHORIZATION, WireMock.equalTo(basicAuth2))
.willReturn(WireMock.aResponse()
.withHeader("X-ClickHouse-Summary",
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}"))
.build());

client.setCredentials(user2, password2);
client.execute("SELECT 1").get();
}
} finally {
mockServer.stop();
}
}

@Test(groups = { "integration" })
public void testJWTWithCloud() throws Exception {
if (!isCloud()) {
Expand All @@ -1011,7 +1064,7 @@ public void testJWTWithCloud() throws Exception {
String jwt = System.getenv("CLIENT_JWT");
Assert.assertTrue(jwt != null && !jwt.trim().isEmpty(), "JWT is missing");
Assert.assertFalse(jwt.contains("\n") || jwt.contains("-----"), "JWT should be single string ready for HTTP header");
try (Client client = newClient().useBearerTokenAuth(jwt).build()) {
try (Client client = newClient().setAccessToken(jwt).build()) {
try {
List<GenericRecord> response = client.queryAll("SELECT user(), now()");
System.out.println("response: " + response.get(0).getString(1) + " time: " + response.get(0).getString(2));
Expand Down
Loading
Loading