Skip to content
Open
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
97 changes: 62 additions & 35 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 @@ -139,11 +140,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);
Object metricsRegistry, Supplier<String> queryIdGenerator, CredentialsManager cManager) {
this.configuration = new ConcurrentHashMap<>(ClientConfigProperties.parseConfigMap(configuration));
this.credentialsManager = cManager;
this.readOnlyConfig = Collections.unmodifiableMap(configuration);
this.metricsRegistry = metricsRegistry;
this.queryIdGenerator = queryIdGenerator;
Expand Down Expand Up @@ -191,7 +194,7 @@

this.httpClientHelper = new HttpAPIClientHelper(this.configuration, metricsRegistry, initSslContext, lz4Factory);
this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown");
this.dbUser = configuration.getOrDefault(ClientConfigProperties.USER.getKey(), ClientConfigProperties.USER.getDefObjVal());
this.dbUser = credentialsManager.getUsername();
this.typeHintMapping = (Map<ClickHouseDataType, Class<?>>) this.configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey());
}

Expand Down Expand Up @@ -340,8 +343,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 @@ -369,13 +375,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 @@ -983,16 +993,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 1003 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 1003 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 @@ -1075,29 +1085,8 @@
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");
}
CredentialsManager cManager = new CredentialsManager(this.configuration);

// Check timezone settings
String useTimeZoneValue = this.configuration.get(ClientConfigProperties.USE_TIMEZONE.getKey());
Expand Down Expand Up @@ -1128,7 +1117,7 @@
}

return new Client(this.endpoints, this.configuration, this.sharedOperationExecutor,
this.columnToMethodMatchingStrategy, this.metricRegistry, this.queryIdGenerator);
this.columnToMethodMatchingStrategy, this.metricRegistry, this.queryIdGenerator, cManager);
}
}

Expand Down Expand Up @@ -2129,8 +2118,46 @@
return unmodifiableDbRolesView;
}

/**
* Updates the credentials used for subsequent requests.
*
* <p>This method is not thread-safe with respect to other credential updates
* or concurrent request execution. Applications must coordinate access if
* they require stronger consistency.
*
* @param username username to use for subsequent requests
* @param password password to use for subsequent requests
*/
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.
*
* <p>This method is not thread-safe with respect to other credential updates
* or concurrent request execution. Applications must coordinate access if
* they require stronger consistency.
*
* @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)}.
*
* <p>This method is not thread-safe with respect to other credential updates
* or concurrent request execution. Applications must coordinate access if
* they require stronger consistency.
*
* @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 @@ -2146,8 +2173,8 @@
* @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 = new HashMap<>(configuration);
credentialsManager.applyCredentials(requestSettings);
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,131 @@
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.
*
* <p>This class is not thread-safe. Callers are responsible for coordinating
* credential updates with request execution if they need stronger consistency.
*/
public class CredentialsManager {
private static final String AUTHORIZATION_HEADER_KEY =
ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION);

private String username;
private String password;
private String accessToken;
private String authorizationHeader;
private boolean useSslAuth;

public CredentialsManager(Map<String, String> configuration) {
validateAuthConfig(configuration);

this.username = configuration.get(ClientConfigProperties.USER.getKey());
this.password = configuration.get(ClientConfigProperties.PASSWORD.getKey());
this.accessToken = readAccessToken(configuration);
this.authorizationHeader = readAuthorizationHeader(configuration, accessToken);
this.useSslAuth = MapUtils.getFlag(configuration, ClientConfigProperties.SSL_AUTH.getKey(), false);
}

public Map<String, Object> snapshot() {
Map<String, Object> snapshot = new HashMap<>();
applyCredentials(snapshot);
return snapshot;
}

public void applyCredentials(Map<String, Object> target) {
putIfNotNull(target, ClientConfigProperties.USER.getKey(), username);
putIfNotNull(target, ClientConfigProperties.PASSWORD.getKey(), password);
putIfNotNull(target, ClientConfigProperties.ACCESS_TOKEN.getKey(), accessToken);
putIfNotNull(target, AUTHORIZATION_HEADER_KEY, authorizationHeader);
if (useSslAuth) {
target.put(ClientConfigProperties.SSL_AUTH.getKey(), Boolean.TRUE);
}
}

/**
* Replaces the current username/password credentials.
*
* <p>This class does not synchronize credential updates. Callers must
* serialize updates and request execution if they require thread safety.
*/
public void setCredentials(String username, String password) {
this.username = username;
this.password = password;
this.useSslAuth = false;
this.accessToken = null;
this.authorizationHeader = null;
}

/**
* Replaces the current credentials with a bearer token.
*
* <p>This class does not synchronize credential updates. Callers must
* serialize updates and request execution if they require thread safety.
*/
public void setAccessToken(String accessToken) {
this.accessToken = accessToken;
this.authorizationHeader = accessToken == null ? null : "Bearer " + accessToken;
this.useSslAuth = false;
this.username = null;
this.password = null;
}

public String getUsername() {
return username == null ? ClientConfigProperties.USER.getDefObjVal() : username;
}

public static void validateAuthConfig(Map<String, ?> configuration) throws ClientMisconfigurationException {
// check if username and password are empty. so can not initiate client?
boolean useSslAuth = MapUtils.getFlag(configuration, ClientConfigProperties.SSL_AUTH.getKey(), false);
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(AUTHORIZATION_HEADER_KEY);

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

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

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

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

private static String readAccessToken(Map<String, String> configuration) {
Object accessToken = configuration.get(ClientConfigProperties.ACCESS_TOKEN.getKey());
if (accessToken == null) {
accessToken = configuration.get(ClientConfigProperties.BEARERTOKEN_AUTH.getKey());
}
return accessToken == null ? null : String.valueOf(accessToken);
}

private static String readAuthorizationHeader(Map<String, String> configuration, String accessToken) {
Object configuredHeader = configuration.get(AUTHORIZATION_HEADER_KEY);
if (configuredHeader != null) {
return String.valueOf(configuredHeader);
}
return accessToken == null ? null : "Bearer " + accessToken;
}

private static void putIfNotNull(Map<String, Object> configuration, String key, Object value) {
if (value != null) {
configuration.put(key, value);
}
}
}
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 @@ -434,6 +434,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 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=AZ1AomRhMKTo52DUqrNN&open=AZ1AomRhMKTo52DUqrNN&pullRequest=2812
String user2 = "client_v2_user2_" + RandomStringUtils.random(8, true, true).toLowerCase();

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=AZ1AomRhMKTo52DUqrNO&open=AZ1AomRhMKTo52DUqrNO&pullRequest=2812
String password1 = "^1A" + RandomStringUtils.random(12, true, true) + "3b$";

Check warning on line 445 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 446 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
Loading
Loading