Skip to content

Commit 4160024

Browse files
committed
Added implementation for Bearer token auth
1 parent 3667f7d commit 4160024

File tree

3 files changed

+125
-9
lines changed

3 files changed

+125
-9
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 49 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -152,7 +152,8 @@ public class Client implements AutoCloseable {
152152
private final ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy;
153153

154154
private Client(Set<String> endpoints, Map<String,String> configuration, boolean useNewImplementation,
155-
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy) {
155+
ExecutorService sharedOperationExecutor, ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy,
156+
Supplier<String> bearerTokenSupplier) {
156157
this.endpoints = endpoints;
157158
this.configuration = configuration;
158159
this.endpoints.forEach(endpoint -> {
@@ -169,7 +170,7 @@ private Client(Set<String> endpoints, Map<String,String> configuration, boolean
169170
}
170171
this.useNewImplementation = useNewImplementation;
171172
if (useNewImplementation) {
172-
this.httpClientHelper = new HttpAPIClientHelper(configuration);
173+
this.httpClientHelper = new HttpAPIClientHelper(configuration, bearerTokenSupplier);
173174
LOG.info("Using new http client implementation");
174175
} else {
175176
this.oldClient = ClientV1AdaptorHelper.createClient(configuration);
@@ -219,6 +220,8 @@ public static class Builder {
219220
private ExecutorService sharedOperationExecutor = null;
220221
private ColumnToMethodMatchingStrategy columnToMethodMatchingStrategy;
221222

223+
private Supplier<String> bearerTokenSupplier = null;
224+
222225
public Builder() {
223226
this.endpoints = new HashSet<>();
224227
this.configuration = new HashMap<String, String>();
@@ -886,6 +889,32 @@ public Builder useHTTPBasicAuth(boolean useBasicAuth) {
886889
return this;
887890
}
888891

892+
/**
893+
* Specifies whether to use Bearer Authentication and what token to use.
894+
* The token will be sent as is, so it should be encoded before passing to this method.
895+
*
896+
* @param bearerToken - token to use
897+
* @return same instance of the builder
898+
*/
899+
public Builder useBearerTokenAuth(String bearerToken) {
900+
this.httpHeader("Authorization", "Bearer " + bearerToken);
901+
return this;
902+
}
903+
904+
/**
905+
* Specifies a supplier for a bearer tokens. It is useful when token should be refreshed.
906+
* Supplier is called each time before sending a request.
907+
* Supplier should return encoded token.
908+
* This configuration cannot be used with {@link #useBearerTokenAuth(String)}.
909+
*
910+
* @param tokenSupplier - token supplier
911+
* @return
912+
*/
913+
public Builder useBearerTokenAuth(Supplier<String> tokenSupplier) {
914+
this.bearerTokenSupplier = tokenSupplier;
915+
return this;
916+
}
917+
889918
public Client build() {
890919
setDefaults();
891920

@@ -896,15 +925,22 @@ public Client build() {
896925
// check if username and password are empty. so can not initiate client?
897926
if (!this.configuration.containsKey("access_token") &&
898927
(!this.configuration.containsKey("user") || !this.configuration.containsKey("password")) &&
899-
!MapUtils.getFlag(this.configuration, "ssl_authentication")) {
900-
throw new IllegalArgumentException("Username and password (or access token, or SSL authentication) are required");
928+
!MapUtils.getFlag(this.configuration, "ssl_authentication", false) &&
929+
!this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") &&
930+
this.bearerTokenSupplier == null) {
931+
throw new IllegalArgumentException("Username and password (or access token or SSL authentication or pre-define Authorization header) are required");
901932
}
902933

903934
if (this.configuration.containsKey("ssl_authentication") &&
904935
(this.configuration.containsKey("password") || this.configuration.containsKey("access_token"))) {
905936
throw new IllegalArgumentException("Only one of password, access token or SSL authentication can be used per client.");
906937
}
907938

939+
if (this.configuration.containsKey(ClientSettings.HTTP_HEADER_PREFIX + "Authorization") &&
940+
this.bearerTokenSupplier != null) {
941+
throw new IllegalArgumentException("Bearer token supplier cannot be used with a predefined Authorization header");
942+
}
943+
908944
if (this.configuration.containsKey("ssl_authentication") &&
909945
!this.configuration.containsKey(ClickHouseClientOption.SSL_CERTIFICATE.getKey())) {
910946
throw new IllegalArgumentException("SSL authentication requires a client certificate");
@@ -943,7 +979,15 @@ public Client build() {
943979
throw new IllegalArgumentException("Nor server timezone nor specific timezone is set");
944980
}
945981

946-
return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor, this.columnToMethodMatchingStrategy);
982+
// check for only new implementation configuration
983+
if (!this.useNewImplementation) {
984+
if (this.bearerTokenSupplier != null) {
985+
throw new IllegalArgumentException("Bearer token supplier cannot be used with old implementation");
986+
}
987+
}
988+
989+
return new Client(this.endpoints, this.configuration, this.useNewImplementation, this.sharedOperationExecutor,
990+
this.columnToMethodMatchingStrategy, this.bearerTokenSupplier);
947991
}
948992

949993
private static final int DEFAULT_NETWORK_BUFFER_SIZE = 300_000;

client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,6 @@
6363
import java.net.NoRouteToHostException;
6464
import java.net.URI;
6565
import java.net.URISyntaxException;
66-
import java.net.URLEncoder;
6766
import java.net.UnknownHostException;
6867
import java.nio.charset.StandardCharsets;
6968
import java.security.NoSuchAlgorithmException;
@@ -74,6 +73,7 @@
7473
import java.util.Set;
7574
import java.util.concurrent.TimeUnit;
7675
import java.util.function.Function;
76+
import java.util.function.Supplier;
7777

7878
public class HttpAPIClientHelper {
7979
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
@@ -90,9 +90,12 @@ public class HttpAPIClientHelper {
9090

9191
private final Set<ClientFaultCause> defaultRetryCauses;
9292

93-
public HttpAPIClientHelper(Map<String, String> configuration) {
93+
private final Supplier<String> bearerTokenSupplier;
94+
95+
public HttpAPIClientHelper(Map<String, String> configuration, Supplier<String> bearerTokenSupplier) {
9496
this.chConfiguration = configuration;
9597
this.httpClient = createHttpClient();
98+
this.bearerTokenSupplier = bearerTokenSupplier;
9699

97100
RequestConfig.Builder reqConfBuilder = RequestConfig.custom();
98101
MapUtils.applyLong(chConfiguration, "connection_request_timeout",
@@ -401,6 +404,8 @@ private void addHeaders(HttpPost req, Map<String, String> chConfig, Map<String,
401404
if (MapUtils.getFlag(chConfig, "ssl_authentication", false)) {
402405
req.addHeader(ClickHouseHttpProto.HEADER_DB_USER, chConfig.get(ClickHouseDefaults.USER.getKey()));
403406
req.addHeader(ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, "on");
407+
} else if (bearerTokenSupplier != null) {
408+
req.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + bearerTokenSupplier.get());
404409
} else if (chConfig.getOrDefault(ClientSettings.HTTP_USE_BASIC_AUTH, "true").equalsIgnoreCase("true")) {
405410
req.addHeader(HttpHeaders.AUTHORIZATION, "Basic " + Base64.getEncoder().encodeToString(
406411
(chConfig.get(ClickHouseDefaults.USER.getKey()) + ":" + chConfig.get(ClickHouseDefaults.PASSWORD.getKey())).getBytes(StandardCharsets.UTF_8)));

client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java

Lines changed: 69 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import java.io.ByteArrayInputStream;
3636
import java.net.Socket;
3737
import java.nio.ByteBuffer;
38+
import java.nio.charset.StandardCharsets;
3839
import java.time.temporal.ChronoUnit;
3940
import java.util.Arrays;
4041
import java.util.Base64;
@@ -44,8 +45,11 @@
4445
import java.util.concurrent.ExecutionException;
4546
import java.util.concurrent.TimeUnit;
4647
import java.util.concurrent.atomic.AtomicInteger;
48+
import java.util.function.Supplier;
4749

4850
import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED;
51+
import static org.junit.Assert.assertEquals;
52+
import static org.junit.Assert.assertThrows;
4953
import static org.junit.Assert.fail;
5054

5155
public class HttpTransportTests extends BaseIntegrationTest {
@@ -59,7 +63,6 @@ public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSocket
5963
ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
6064

6165
int proxyPort = new Random().nextInt(1000) + 10000;
62-
System.out.println("proxyPort: " + proxyPort);
6366
ConnectionCounterListener connectionCounter = new ConnectionCounterListener();
6467
WireMockServer proxy = new WireMockServer(WireMockConfiguration
6568
.options().port(proxyPort)
@@ -147,7 +150,6 @@ public void closed(Socket socket) {
147150
public void testConnectionRequestTimeout() {
148151

149152
int serverPort = new Random().nextInt(1000) + 10000;
150-
System.out.println("proxyPort: " + serverPort);
151153
ConnectionCounterListener connectionCounter = new ConnectionCounterListener();
152154
WireMockServer proxy = new WireMockServer(WireMockConfiguration
153155
.options().port(serverPort)
@@ -638,4 +640,69 @@ public void testErrorWithSendProgressHeaders() throws Exception {
638640
}
639641
}
640642
}
643+
644+
@Test(groups = { "integration" })
645+
public void testBearerTokenAuth() throws Exception {
646+
WireMockServer mockServer = new WireMockServer( WireMockConfiguration
647+
.options().port(9090).notifier(new ConsoleNotifier(false)));
648+
mockServer.start();
649+
650+
String jwtToken1 = Arrays.stream(
651+
new String[]{"header", "payload", "signature"})
652+
.map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8)))
653+
.reduce((s1, s2) -> s1 + "." + s2).get();
654+
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
655+
.useBearerTokenAuth(jwtToken1)
656+
.build()) {
657+
658+
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
659+
.withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1))
660+
.willReturn(WireMock.aResponse()
661+
.withHeader("X-ClickHouse-Summary",
662+
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build());
663+
664+
try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) {
665+
Assert.assertEquals(response.getReadBytes(), 10);
666+
} catch (Exception e) {
667+
Assert.fail("Unexpected exception", e);
668+
}
669+
}
670+
671+
String jwtToken2 = Arrays.stream(
672+
new String[]{"header2", "payload2", "signature2"})
673+
.map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8)))
674+
.reduce((s1, s2) -> s1 + "." + s2).get();
675+
final AtomicInteger callCount = new AtomicInteger(0);
676+
Supplier<String> tokenSupplier = () -> {
677+
callCount.incrementAndGet();
678+
return jwtToken2;
679+
};
680+
681+
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
682+
.withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2))
683+
.willReturn(WireMock.aResponse()
684+
.withHeader("X-ClickHouse-Summary",
685+
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build());
686+
687+
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
688+
.useBearerTokenAuth(tokenSupplier)
689+
.build()) {
690+
691+
for (int i = 0; i < 3; i++ ) {
692+
693+
try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) {
694+
Assert.assertEquals(response.getReadBytes(), 10);
695+
} catch (Exception e) {
696+
Assert.fail("Unexpected exception", e);
697+
}
698+
}
699+
}
700+
701+
assertEquals(callCount.get(), 3);
702+
703+
assertThrows(IllegalArgumentException.class, () -> {
704+
new Client.Builder().useBearerTokenAuth("token")
705+
.useBearerTokenAuth(() -> "token2").build();
706+
});
707+
}
641708
}

0 commit comments

Comments
 (0)