From 416002402bee7c732a30710515ed9ab6b1b7cac8 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 4 Nov 2024 14:02:55 -0800 Subject: [PATCH 1/4] Added implementation for Bearer token auth --- .../com/clickhouse/client/api/Client.java | 54 ++++++++++++-- .../api/internal/HttpAPIClientHelper.java | 9 ++- .../clickhouse/client/HttpTransportTests.java | 71 ++++++++++++++++++- 3 files changed, 125 insertions(+), 9 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index ea887916b..abe69e31b 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -152,7 +152,8 @@ public class Client implements AutoCloseable { private final ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; private Client(Set endpoints, Map configuration, boolean useNewImplementation, - ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy) { + ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, + Supplier bearerTokenSupplier) { this.endpoints = endpoints; this.configuration = configuration; this.endpoints.forEach(endpoint -> { @@ -169,7 +170,7 @@ private Client(Set endpoints, Map configuration, boolean } this.useNewImplementation = useNewImplementation; if (useNewImplementation) { - this.httpClientHelper = new HttpAPIClientHelper(configuration); + this.httpClientHelper = new HttpAPIClientHelper(configuration, bearerTokenSupplier); LOG.info("Using new http client implementation"); } else { this.oldClient = ClientV1AdaptorHelper.createClient(configuration); @@ -219,6 +220,8 @@ public static class Builder { private ExecutorService sharedOperationExecutor = null; private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; + private Supplier bearerTokenSupplier = null; + public Builder() { this.endpoints = new HashSet<>(); this.configuration = new HashMap(); @@ -886,6 +889,32 @@ public Builder useHTTPBasicAuth(boolean useBasicAuth) { return this; } + /** + * 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. + * + * @param bearerToken - token to use + * @return same instance of the builder + */ + public Builder useBearerTokenAuth(String bearerToken) { + this.httpHeader("Authorization", "Bearer " + bearerToken); + return this; + } + + /** + * Specifies a supplier for a bearer tokens. It is useful when token should be refreshed. + * Supplier is called each time before sending a request. + * Supplier should return encoded token. + * This configuration cannot be used with {@link #useBearerTokenAuth(String)}. + * + * @param tokenSupplier - token supplier + * @return + */ + public Builder useBearerTokenAuth(Supplier tokenSupplier) { + this.bearerTokenSupplier = tokenSupplier; + return this; + } + public Client build() { setDefaults(); @@ -896,8 +925,10 @@ public Client build() { // check if username and password are empty. so can not initiate client? if (!this.configuration.containsKey("access_token") && (!this.configuration.containsKey("user") || !this.configuration.containsKey("password")) && - !MapUtils.getFlag(this.configuration, "ssl_authentication")) { - throw new IllegalArgumentException("Username and password (or access token, or SSL authentication) are required"); + !MapUtils.getFlag(this.configuration, "ssl_authentication", false) && + !this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") && + this.bearerTokenSupplier == null) { + throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required"); } if (this.configuration.containsKey("ssl_authentication") && @@ -905,6 +936,11 @@ public Client build() { throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client."); } + if (this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") && + this.bearerTokenSupplier != null) { + throw new IllegalArgumentException("Bearer token supplier cannot be used with a predefined Authorization header"); + } + if (this.configuration.containsKey("ssl_authentication") && !this.configuration.containsKey(ClickHouseClientOption.SSL_CERTIFICATE.getKey())) { throw new IllegalArgumentException("SSL authentication requires a client certificate"); @@ -943,7 +979,15 @@ public Client build() { throw new IllegalArgumentException("Nor server timezone nor specific timezone is set"); } - return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor, this.columnToMethodMatchingStrategy); + // check for only new implementation configuration + if (!this.useNewImplementation) { + if (this.bearerTokenSupplier != null) { + throw new IllegalArgumentException("Bearer token supplier cannot be used with old implementation"); + } + } + + return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor, + this.columnToMethodMatchingStrategy, this.bearerTokenSupplier); } private static final int DEFAULT_NETWORK_BUFFER_SIZE = 300_000; diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index 1569af01d..ad6c9126d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -63,7 +63,6 @@ import java.net.NoRouteToHostException; import java.net.URI; import java.net.URISyntaxException; -import java.net.URLEncoder; import java.net.UnknownHostException; import java.nio.charset.StandardCharsets; import java.security.NoSuchAlgorithmException; @@ -74,6 +73,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import java.util.function.Supplier; public class HttpAPIClientHelper { private static final Logger LOG = LoggerFactory.getLogger(Client.class); @@ -90,9 +90,12 @@ public class HttpAPIClientHelper { private final Set defaultRetryCauses; - public HttpAPIClientHelper(Map configuration) { + private final Supplier bearerTokenSupplier; + + public HttpAPIClientHelper(Map configuration, Supplier bearerTokenSupplier) { this.chConfiguration = configuration; this.httpClient = createHttpClient(); + this.bearerTokenSupplier = bearerTokenSupplier; RequestConfig.Builder reqConfBuilder = RequestConfig.custom(); MapUtils.applyLong(chConfiguration, "connection_request_timeout", @@ -401,6 +404,8 @@ private void addHeaders(HttpPost req, Map chConfig, Map 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) + .build()) { + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + String jwtToken2 = Arrays.stream( + new String[]{"header2", "payload2", "signature2"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); + final AtomicInteger callCount = new AtomicInteger(0); + Supplier tokenSupplier = () -> { + callCount.incrementAndGet(); + return jwtToken2; + }; + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(tokenSupplier) + .build()) { + + for (int i = 0; i < 3; i++ ) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + } + + assertEquals(callCount.get(), 3); + + assertThrows(IllegalArgumentException.class, () -> { + new Client.Builder().useBearerTokenAuth("token") + .useBearerTokenAuth(() -> "token2").build(); + }); + } } From 83536898c6dd2704dc24178145a9b311e89dd81b Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 2 Dec 2024 11:45:01 -0800 Subject: [PATCH 2/4] fixed after merge --- client-v2/src/main/java/com/clickhouse/client/api/Client.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index a8b1b7c59..2f2199e91 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -980,7 +980,7 @@ public Client build() { if (!this.configuration.containsKey("access_token") && (!this.configuration.containsKey("user") || !this.configuration.containsKey("password")) && !MapUtils.getFlag(this.configuration, "ssl_authentication", false) && - !this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") && + !this.configuration.containsKey(ClientConfigProperties.HTTP_HEADER_PREFIX + "Authorization") && this.bearerTokenSupplier == null) { throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required"); } @@ -990,7 +990,7 @@ public Client build() { throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client."); } - if (this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") && + if (this.configuration.containsKey(ClientConfigProperties.HTTP_HEADER_PREFIX + "Authorization") && this.bearerTokenSupplier != null) { throw new IllegalArgumentException("Bearer token supplier cannot be used with a predefined Authorization header"); } From 471376346d99fb63f883128f1ea877f953d0acfc Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 12 Dec 2024 12:38:17 -0800 Subject: [PATCH 3/4] added method to update bearer token --- .../com/clickhouse/client/api/Client.java | 56 ++++++------------- .../client/api/ClientConfigProperties.java | 5 ++ .../client/api/ServerException.java | 22 ++++++++ .../api/internal/HttpAPIClientHelper.java | 18 +++--- .../clickhouse/client/HttpTransportTests.java | 48 ++++++++-------- 5 files changed, 77 insertions(+), 72 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index a1b28a1fb..c5a9d053c 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -117,7 +117,6 @@ * ... * } * } - * * } * * @@ -131,6 +130,9 @@ public class Client implements AutoCloseable { private final Set endpoints; private final Map configuration; + + private final Map readOnlyConfig; + private final List serverNodes = new ArrayList<>(); // POJO serializer mapping (class -> (schema -> (format -> serializer))) @@ -154,10 +156,10 @@ public class Client implements AutoCloseable { private final ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; private Client(Set endpoints, Map configuration, boolean useNewImplementation, - ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy, - Supplier bearerTokenSupplier) { + ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy) { this.endpoints = endpoints; this.configuration = configuration; + this.readOnlyConfig = Collections.unmodifiableMap(this.configuration); this.endpoints.forEach(endpoint -> { this.serverNodes.add(ClickHouseNode.of(endpoint, this.configuration)); }); @@ -172,7 +174,7 @@ private Client(Set endpoints, Map configuration, boolean } this.useNewImplementation = useNewImplementation; if (useNewImplementation) { - this.httpClientHelper = new HttpAPIClientHelper(configuration, bearerTokenSupplier); + this.httpClientHelper = new HttpAPIClientHelper(configuration); LOG.info("Using new http client implementation"); } else { this.oldClient = ClientV1AdaptorHelper.createClient(configuration); @@ -226,8 +228,6 @@ public static class Builder { private ExecutorService sharedOperationExecutor = null; private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy; - private Supplier bearerTokenSupplier = null; - public Builder() { this.endpoints = new HashSet<>(); this.configuration = new HashMap(); @@ -855,7 +855,7 @@ public Builder allowBinaryReaderToReuseBuffers(boolean reuse) { * @return same instance of the builder */ public Builder httpHeader(String key, String value) { - this.configuration.put(ClientConfigProperties.HTTP_HEADER_PREFIX + key.toUpperCase(Locale.US), value); + this.configuration.put(ClientConfigProperties.httpHeader(key), value); return this; } @@ -866,7 +866,7 @@ public Builder httpHeader(String key, String value) { * @return same instance of the builder */ public Builder httpHeader(String key, Collection values) { - this.configuration.put(ClientConfigProperties.HTTP_HEADER_PREFIX + key.toUpperCase(Locale.US), ClientConfigProperties.commaSeparated(values)); + this.configuration.put(ClientConfigProperties.httpHeader(key), ClientConfigProperties.commaSeparated(values)); return this; } @@ -965,21 +965,8 @@ public Builder setOptions(Map options) { * @return same instance of the builder */ public Builder useBearerTokenAuth(String bearerToken) { - this.httpHeader("Authorization", "Bearer " + bearerToken); - return this; - } - - /** - * Specifies a supplier for a bearer tokens. It is useful when token should be refreshed. - * Supplier is called each time before sending a request. - * Supplier should return encoded token. - * This configuration cannot be used with {@link #useBearerTokenAuth(String)}. - * - * @param tokenSupplier - token supplier - * @return - */ - public Builder useBearerTokenAuth(Supplier tokenSupplier) { - this.bearerTokenSupplier = tokenSupplier; + // Most JWT libraries (https://jwt.io/libraries?language=Java) compact tokens in proper way + this.httpHeader(HttpHeaders.AUTHORIZATION, "Bearer " + bearerToken); return this; } @@ -994,8 +981,7 @@ public Client build() { if (!this.configuration.containsKey("access_token") && (!this.configuration.containsKey("user") || !this.configuration.containsKey("password")) && !MapUtils.getFlag(this.configuration, "ssl_authentication", false) && - !this.configuration.containsKey(ClientConfigProperties.HTTP_HEADER_PREFIX + "Authorization") && - this.bearerTokenSupplier == null) { + !this.configuration.containsKey(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION))) { throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required"); } @@ -1004,11 +990,6 @@ public Client build() { throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client."); } - if (this.configuration.containsKey(ClientConfigProperties.HTTP_HEADER_PREFIX + "Authorization") && - this.bearerTokenSupplier != null) { - throw new IllegalArgumentException("Bearer token supplier cannot be used with a predefined Authorization header"); - } - if (this.configuration.containsKey("ssl_authentication") && !this.configuration.containsKey(ClientConfigProperties.SSL_CERTIFICATE.getKey())) { throw new IllegalArgumentException("SSL authentication requires a client certificate"); @@ -1047,15 +1028,8 @@ public Client build() { throw new IllegalArgumentException("Nor server timezone nor specific timezone is set"); } - // check for only new implementation configuration - if (!this.useNewImplementation) { - if (this.bearerTokenSupplier != null) { - throw new IllegalArgumentException("Bearer token supplier cannot be used with old implementation"); - } - } - return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor, - this.columnToMethodMatchingStrategy, this.bearerTokenSupplier); + this.columnToMethodMatchingStrategy); } private static final int DEFAULT_NETWORK_BUFFER_SIZE = 300_000; @@ -2147,7 +2121,7 @@ public String toString() { * @return - configuration options */ public Map getConfiguration() { - return Collections.unmodifiableMap(configuration); + return readOnlyConfig; } /** Returns operation timeout in seconds */ @@ -2194,6 +2168,10 @@ public Collection getDBRoles() { return unmodifiableDbRolesView; } + public void updateBearerToken(String bearer) { + this.configuration.put(ClientConfigProperties.httpHeader(HttpHeaders.AUTHORIZATION), "Bearer " + bearer); + } + private ClickHouseNode getNextAliveNode() { return serverNodes.get(0); } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java index 374757597..6a0e30af7 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java @@ -4,6 +4,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.stream.Collectors; /** @@ -157,6 +158,10 @@ public static String serverSetting(String key) { return SERVER_SETTING_PREFIX + key; } + public static String httpHeader(String key) { + return HTTP_HEADER_PREFIX + key.toUpperCase(Locale.US); + } + public static String commaSeparated(Collection values) { StringBuilder sb = new StringBuilder(); for (Object value : values) { diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ServerException.java b/client-v2/src/main/java/com/clickhouse/client/api/ServerException.java index 6e2c0f558..c4476e1cc 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/ServerException.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/ServerException.java @@ -7,12 +7,34 @@ public class ServerException extends RuntimeException { public static final int TABLE_NOT_FOUND = 60; private final int code; + + private final int transportProtocolCode; + public ServerException(int code, String message) { + this(code, message, 500); + } + + public ServerException(int code, String message, int transportProtocolCode) { super(message); this.code = code; + this.transportProtocolCode = transportProtocolCode; } + /** + * Returns CH server error code. May return 0 if code is unknown. + * @return - error code from server response + */ public int getCode() { return code; } + + /** + * Returns error code of underlying transport protocol. For example, HTTP status. + * By default, will return {@code 500 } what is derived from HTTP Server Internal Error. + * + * @return - transport status code + */ + public int getTransportProtocolCode() { + return transportProtocolCode; + } } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index 4119a1f24..aa27f3746 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -94,12 +94,9 @@ public class HttpAPIClientHelper { private String httpClientUserAgentPart; - private final Supplier bearerTokenSupplier; - - public HttpAPIClientHelper(Map configuration, Supplier bearerTokenSupplier) { + public HttpAPIClientHelper(Map configuration) { this.chConfiguration = configuration; this.httpClient = createHttpClient(); - this.bearerTokenSupplier = bearerTokenSupplier; this.httpClientUserAgentPart = this.httpClient.getClass().getPackage().getImplementationTitle() + "/" + this.httpClient.getClass().getPackage().getImplementationVersion(); @@ -339,10 +336,13 @@ public Exception readError(ClassicHttpResponse httpResponse) { String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") .replaceAll("\\\\/", "/"); - return new ServerException(serverCode, msg); + if (msg.trim().isEmpty()) { + msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; + } + return new ServerException(serverCode, msg, httpResponse.getCode()); } catch (Exception e) { LOG.error("Failed to read error message", e); - return new ServerException(serverCode, String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " "); + return new ServerException(serverCode, String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")", httpResponse.getCode()); } } @@ -427,8 +427,6 @@ private void addHeaders(HttpPost req, Map chConfig, Map chConfig, Map entry : chConfig.entrySet()) { if (entry.getKey().startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { - req.addHeader(entry.getKey().substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), entry.getValue()); + req.setHeader(entry.getKey().substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), entry.getValue()); } } for (Map.Entry entry : requestConfig.entrySet()) { if (entry.getKey().startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { - req.addHeader(entry.getKey().substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), entry.getValue().toString()); + req.setHeader(entry.getKey().substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), entry.getValue().toString()); } } diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index bb80c9a9c..54f581ae8 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -809,6 +809,7 @@ public void testBearerTokenAuth() throws Exception { .reduce((s1, s2) -> s1 + "." + s2).get(); try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) .build()) { mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) @@ -828,37 +829,38 @@ public void testBearerTokenAuth() throws Exception { new String[]{"header2", "payload2", "signature2"}) .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) .reduce((s1, s2) -> s1 + "." + s2).get(); - final AtomicInteger callCount = new AtomicInteger(0); - Supplier tokenSupplier = () -> { - callCount.incrementAndGet(); - return jwtToken2; - }; - + + mockServer.resetAll(); mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + .withStatus(HttpStatus.SC_UNAUTHORIZED)) + .build()); try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(tokenSupplier) + .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) .build()) { - for (int i = 0; i < 3; i++ ) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } + try { + client.execute("SELECT 1").get(); + fail("Exception expected"); + } catch (ServerException e) { + Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); } - } - assertEquals(callCount.get(), 3); + mockServer.resetAll(); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) - assertThrows(IllegalArgumentException.class, () -> { - new Client.Builder().useBearerTokenAuth("token") - .useBearerTokenAuth(() -> "token2").build(); - }); + .build()); + + client.updateBearerToken(jwtToken2); + + client.execute("SELECT 1").get(); + } } } From cb4c32dbf34c8f574b5e4165a1bb0166ea98eebf Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 17 Dec 2024 23:17:45 -0800 Subject: [PATCH 4/4] fixed tests by stopping mock server --- .../clickhouse/client/HttpTransportTests.java | 100 +++++++++--------- 1 file changed, 52 insertions(+), 48 deletions(-) diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index 54f581ae8..c7badb6d2 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -803,64 +803,68 @@ public void testBearerTokenAuth() throws Exception { .options().port(9090).notifier(new ConsoleNotifier(false))); mockServer.start(); - String jwtToken1 = Arrays.stream( - new String[]{"header", "payload", "signature"}) - .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) - .compressServerResponse(false) - .build()) { - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + try { + String jwtToken1 = Arrays.stream( + new String[]{"header", "payload", "signature"}) + .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) + .compressServerResponse(false) + .build()) { - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Unexpected exception", e); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } } - } - - String jwtToken2 = Arrays.stream( - new String[]{"header2", "payload2", "signature2"}) - .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) - .reduce((s1, s2) -> s1 + "." + s2).get(); - - mockServer.resetAll(); - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_UNAUTHORIZED)) - .build()); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(jwtToken1) - .compressServerResponse(false) - .build()) { - - try { - client.execute("SELECT 1").get(); - fail("Exception expected"); - } catch (ServerException e) { - Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); - } + String jwtToken2 = Arrays.stream( + new String[]{"header2", "payload2", "signature2"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); mockServer.resetAll(); mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) - + .withStatus(HttpStatus.SC_UNAUTHORIZED)) .build()); - client.updateBearerToken(jwtToken2); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(jwtToken1) + .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("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) + + .build()); - client.execute("SELECT 1").get(); + client.updateBearerToken(jwtToken2); + + client.execute("SELECT 1").get(); + } + } finally { + mockServer.stop(); } } }