Skip to content

Commit

Permalink
Add cached token option to jdbc externalAuthentication
Browse files Browse the repository at this point in the history
This change allows sharing external authentication tokens
between different Connections. Each time when a new token
is required, first Connection that needs it, will handle obtaining
a new token when all the other Connections wait for this operation
to finish.

Token is kept in memmory, guarded by ReadWriteLock.

To enable token cache use externalAuthenticationTokenCache=MEMORY
Default value for externalAuthenticationTokenCache is NONE.
  • Loading branch information
s2lomon authored and kokosing committed Apr 9, 2021
1 parent fe5335b commit ceb2d1b
Show file tree
Hide file tree
Showing 12 changed files with 475 additions and 25 deletions.
2 changes: 2 additions & 0 deletions client/trino-cli/src/main/java/io/trino/cli/QueryRunner.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.trino.client.StatementClient;
import io.trino.client.auth.external.ExternalAuthenticator;
import io.trino.client.auth.external.HttpTokenPoller;
import io.trino.client.auth.external.KnownToken;
import io.trino.client.auth.external.RedirectHandler;
import io.trino.client.auth.external.TokenPoller;
import okhttp3.OkHttpClient;
Expand Down Expand Up @@ -193,6 +194,7 @@ private static void setupExternalAuth(
ExternalAuthenticator authenticator = new ExternalAuthenticator(
redirectHandler,
poller,
KnownToken.local(),
Duration.ofMinutes(10));

builder.authenticator(authenticator);
Expand Down
6 changes: 6 additions & 0 deletions client/trino-client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,12 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>concurrent</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.squareup.okhttp3</groupId>
<artifactId>mockwebserver</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,41 +44,41 @@ public class ExternalAuthenticator
private final TokenPoller tokenPoller;
private final RedirectHandler redirectHandler;
private final Duration timeout;
private Token knownToken;
private final KnownToken knownToken;

public ExternalAuthenticator(RedirectHandler redirect, TokenPoller tokenPoller, Duration timeout)
public ExternalAuthenticator(RedirectHandler redirect, TokenPoller tokenPoller, KnownToken knownToken, Duration timeout)
{
this.tokenPoller = requireNonNull(tokenPoller, "tokenPoller is null");
this.redirectHandler = requireNonNull(redirect, "redirect is null");
this.knownToken = requireNonNull(knownToken, "knownToken is null");
this.timeout = requireNonNull(timeout, "timeout is null");
}

@Nullable
@Override
public Request authenticate(Route route, Response response)
{
knownToken = null;

Optional<ExternalAuthentication> authentication = toAuthentication(response);
if (!authentication.isPresent()) {
return null;
}
knownToken.setupToken(() -> {
Optional<ExternalAuthentication> authentication = toAuthentication(response);
if (!authentication.isPresent()) {
return Optional.empty();
}

Optional<Token> token = authentication.get().obtainToken(timeout, redirectHandler, tokenPoller);
if (!token.isPresent()) {
return null;
}
return authentication.get().obtainToken(timeout, redirectHandler, tokenPoller);
});

knownToken = token.get();
return withBearerToken(response.request(), knownToken);
return knownToken.getToken()
.map(token -> withBearerToken(response.request(), token))
.orElse(null);
}

@Override
public Response intercept(Chain chain)
throws IOException
{
if (knownToken != null) {
return chain.proceed(withBearerToken(chain.request(), knownToken));
Optional<Token> token = knownToken.getToken();
if (token.isPresent()) {
return chain.proceed(withBearerToken(chain.request(), token.get()));
}

return chain.proceed(chain.request());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.client.auth.external;

import java.util.Optional;
import java.util.function.Supplier;

public interface KnownToken
{
Optional<Token> getToken();

void setupToken(Supplier<Optional<Token>> tokenSource);

static KnownToken local()
{
return new LocalKnownToken();
}

static KnownToken memoryCached()
{
return MemoryCachedKnownToken.INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.client.auth.external;

import javax.annotation.concurrent.NotThreadSafe;

import java.util.Optional;
import java.util.function.Supplier;

import static java.util.Objects.requireNonNull;

/**
* LocalKnownToken class keeps the token on its field
* and it's designed to use it in fully serialized manner.
*/
@NotThreadSafe
class LocalKnownToken
implements KnownToken
{
private Optional<Token> knownToken = Optional.empty();

@Override
public Optional<Token> getToken()
{
return knownToken;
}

@Override
public void setupToken(Supplier<Optional<Token>> tokenSource)
{
requireNonNull(tokenSource, "tokenSource is null");

knownToken = tokenSource.get();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.client.auth.external;

import javax.annotation.concurrent.ThreadSafe;

import java.util.Optional;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;

/**
* This KnownToken instance forces all Connections to reuse same token.
* Every time an existing token is considered to be invalid each Connection
* will try to obtain a new token, but only the first one will actually do the job,
* where every other connection will be waiting on readLock
* until obtaining new token finishes.
* <p>
* In general the game is to reuse same token and obtain it only once, no matter how
* many Connections will be actively using it. It's very important as obtaining the new token
* will take minutes, as it mostly requires user thinking time.
*/
@ThreadSafe
class MemoryCachedKnownToken
implements KnownToken
{
public static final MemoryCachedKnownToken INSTANCE = new MemoryCachedKnownToken();

private final ReadWriteLock lock = new ReentrantReadWriteLock();
private final Lock readLock = lock.readLock();
private final Lock writeLock = lock.writeLock();
private Optional<Token> knownToken = Optional.empty();

private MemoryCachedKnownToken()
{
}

@Override
public Optional<Token> getToken()
{
try {
readLock.lockInterruptibly();
return knownToken;
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
finally {
readLock.unlock();
}
}

@Override
public void setupToken(Supplier<Optional<Token>> tokenSource)
{
// Try to lock and generate new token. If some other thread (Connection) has
// already obtained writeLock and is generating new token, then skipp this
// to block on getToken()
if (writeLock.tryLock()) {
try {
// Clear knownToken before obtaining new token, as it might fail leaving old invalid token.
knownToken = Optional.empty();
knownToken = tokenSource.get();
}
finally {
writeLock.unlock();
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,21 +14,46 @@
package io.trino.client.auth.external;

import java.net.URI;
import java.time.Duration;
import java.util.concurrent.atomic.AtomicInteger;

public class MockRedirectHandler
implements RedirectHandler
{
private URI redirectedTo;
private AtomicInteger redirectionCount = new AtomicInteger(0);
private Duration redirectTime;

@Override
public void redirectTo(URI uri)
throws RedirectException
{
redirectedTo = uri;
redirectionCount.incrementAndGet();
try {
if (redirectTime != null) {
Thread.sleep(redirectTime.toMillis());
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}

public URI redirectedTo()
{
return redirectedTo;
}

public int getRedirectionCount()
{
return redirectionCount.get();
}

public MockRedirectHandler sleepOnRedirect(Duration redirectTime)
{
this.redirectTime = redirectTime;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,21 +17,21 @@

import java.net.URI;
import java.time.Duration;
import java.util.ArrayDeque;
import java.util.HashMap;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.BlockingDeque;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingDeque;

public final class MockTokenPoller
implements TokenPoller
{
private final Map<URI, Queue<TokenPollResult>> results = new HashMap<>();
private final Map<URI, BlockingDeque<TokenPollResult>> results = new ConcurrentHashMap<>();

public MockTokenPoller withResult(URI tokenUri, TokenPollResult result)
{
results.compute(tokenUri, (uri, queue) -> {
if (queue == null) {
return new ArrayDeque<>(ImmutableList.of(result));
return new LinkedBlockingDeque<>(ImmutableList.of(result));
}
queue.add(result);
return queue;
Expand All @@ -42,7 +42,7 @@ public MockTokenPoller withResult(URI tokenUri, TokenPollResult result)
@Override
public TokenPollResult pollForToken(URI tokenUri, Duration ignored)
{
Queue<TokenPollResult> queue = results.get(tokenUri);
BlockingDeque<TokenPollResult> queue = results.get(tokenUri);
if (queue == null) {
throw new IllegalArgumentException("Unknown token URI: " + tokenUri);
}
Expand Down
Loading

0 comments on commit ceb2d1b

Please sign in to comment.