Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-21108][rest-client] Add basic & digest auth support to rest client #15365

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
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
Original file line number Diff line number Diff line change
Expand Up @@ -166,4 +166,16 @@ public class RestOptions {
"Thread priority of the REST server's executor for processing asynchronous requests. "
+ "Lowering the thread priority will give Flink's main components more CPU time whereas "
+ "increasing will allocate more time for the REST server's processing.");

@Documentation.Section(Documentation.Sections.EXPERT_REST)
public static final ConfigOption<String> CLIENT_AUTH_USERNAME =
key("rest.client.auth.username")
.noDefaultValue()
.withDescription("Rest client authentication user name.");

@Documentation.Section(Documentation.Sections.EXPERT_REST)
public static final ConfigOption<String> CLIENT_AUTH_PASSWORD =
key("rest.client.auth.password")
.noDefaultValue()
.withDescription("Rest client authentication password.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,4 +50,19 @@ public void testBindAddressSecondDeprecatedKey() {

assertThat(actualAddress, is(equalTo(expectedAddress)));
}

@Test
public void testBasicAuthFallbackKeys() {
final Configuration configuration = new Configuration();
final String expectedUserName = "auth_user";
final String expectedPassword = "auth_password";
configuration.setString(RestOptions.CLIENT_AUTH_USERNAME, expectedUserName);
configuration.setString(RestOptions.CLIENT_AUTH_PASSWORD, expectedPassword);

final String actualUserName = configuration.getString(RestOptions.CLIENT_AUTH_USERNAME);
final String actualPassword = configuration.getString(RestOptions.CLIENT_AUTH_PASSWORD);

assertThat(actualUserName, is(equalTo(expectedUserName)));
assertThat(actualPassword, is(equalTo(expectedPassword)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ public class HistoryServer {
private final HistoryServerArchiveFetcher archiveFetcher;

@Nullable private final SSLHandlerFactory serverSSLFactory;

private WebFrontendBootstrap netty;

private final Object startupShutdownLock = new Object();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,16 @@ private static <T> void retryOperation(
"Operation future was cancelled.", throwable));
} else {
throwable = ExceptionUtils.stripExecutionException(throwable);
if (!retryPredicate.test(throwable)) {
if (ExceptionUtils.findThrowable(
throwable, RestartableException.class)
.isPresent()) {
retryOperation(
resultFuture,
operation,
retries,
retryPredicate,
executor);
} else if (!retryPredicate.test(throwable)) {
resultFuture.completeExceptionally(
new RetryException(
"Stopped retrying the operation because the error is not "
Expand Down Expand Up @@ -360,7 +369,26 @@ private static <T> void retryOperationWithDelay(
"Operation future was cancelled.", throwable));
} else {
throwable = ExceptionUtils.stripExecutionException(throwable);
if (!retryPredicate.test(throwable)) {
if (ExceptionUtils.findThrowable(
throwable, RestartableException.class)
.isPresent()) {
final ScheduledFuture<?> scheduledFuture =
scheduledExecutor.schedule(
(Runnable)
() ->
retryOperationWithDelay(
resultFuture,
operation,
retryStrategy,
retryPredicate,
scheduledExecutor),
0,
TimeUnit.MILLISECONDS);

resultFuture.whenComplete(
(innerT, innerThrowable) ->
scheduledFuture.cancel(false));
} else if (!retryPredicate.test(throwable)) {
resultFuture.completeExceptionally(throwable);
} else if (retryStrategy.getNumRemainingRetries() > 0) {
long retryDelayMillis =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.flink.runtime.concurrent;

import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;

import java.util.Optional;

/** An exception that is thrown if the failure of a operation can be restarted immediately. */
public class RestartableException extends FlinkException {
private static final long serialVersionUID = 8787131901729148561L;

private final Optional token;

public RestartableException(String message, Optional token) {
super(message);
this.token = Preconditions.checkNotNull(token);
}

public Optional getToken() {
return token;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.concurrent.RestartableException;
import org.apache.flink.runtime.io.network.netty.SSLHandlerFactory;
import org.apache.flink.runtime.rest.auth.RestClientAuth;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
Expand Down Expand Up @@ -88,13 +90,17 @@
import java.nio.file.Path;
import java.util.Collection;
import java.util.Collections;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;

import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.PROXY_AUTHENTICATION_REQUIRED;
import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE;
import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;

/** This client is the counter-part to the {@link RestServerEndpoint}. */
public class RestClient implements AutoCloseableAsync {
Expand All @@ -111,10 +117,13 @@ public class RestClient implements AutoCloseableAsync {

private final AtomicBoolean isRunning = new AtomicBoolean(true);

private final Optional<RestClientAuth> auth;

public RestClient(RestClientConfiguration configuration, Executor executor) {
Preconditions.checkNotNull(configuration);
this.executor = Preconditions.checkNotNull(executor);
this.terminationFuture = new CompletableFuture<>();
this.auth = configuration.getAuth();

final SSLHandlerFactory sslHandlerFactory = configuration.getSslHandlerFactory();
ChannelInitializer<SocketChannel> initializer =
Expand Down Expand Up @@ -146,7 +155,7 @@ protected void initChannel(SocketChannel socketChannel) {
configuration.getIdlenessTimeout(),
configuration.getIdlenessTimeout(),
TimeUnit.MILLISECONDS))
.addLast(new ClientHandler());
.addLast(new ClientHandler(auth));
} catch (Throwable t) {
t.printStackTrace();
ExceptionUtils.rethrow(t);
Expand Down Expand Up @@ -324,7 +333,8 @@ CompletableFuture<P> sendRequest(
targetUrl,
messageHeaders.getHttpMethod().getNettyHttpMethod(),
payload,
fileUploads);
fileUploads,
auth);

final JavaType responseType;

Expand All @@ -344,21 +354,29 @@ CompletableFuture<P> sendRequest(
return submitRequest(targetAddress, targetPort, httpRequest, responseType);
}

private static HttpHeaders addAuthorization(
final Optional<RestClientAuth> auth,
final HttpMethod method,
final String uri,
final HttpHeaders headers) {
return auth.map(a -> a.addAuthorization(method, uri, headers)).orElse(headers);
}

private static Request createRequest(
String targetAddress,
String targetUrl,
HttpMethod httpMethod,
ByteBuf jsonPayload,
Collection<FileUpload> fileUploads)
Collection<FileUpload> fileUploads,
Optional<RestClientAuth> auth)
throws IOException {
if (fileUploads.isEmpty()) {

HttpRequest httpRequest =
new DefaultFullHttpRequest(
HttpVersion.HTTP_1_1, httpMethod, targetUrl, jsonPayload);

httpRequest
.headers()
addAuthorization(auth, httpMethod, targetUrl, httpRequest.headers())
.set(HttpHeaders.Names.HOST, targetAddress)
.set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE)
.add(HttpHeaders.Names.CONTENT_LENGTH, jsonPayload.capacity())
Expand All @@ -369,8 +387,7 @@ private static Request createRequest(
HttpRequest httpRequest =
new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, httpMethod, targetUrl);

httpRequest
.headers()
addAuthorization(auth, httpMethod, targetUrl, httpRequest.headers())
.set(HttpHeaders.Names.HOST, targetAddress)
.set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE);

Expand Down Expand Up @@ -549,37 +566,57 @@ private static class ClientHandler extends SimpleChannelInboundHandler<Object> {

private final CompletableFuture<JsonResponse> jsonFuture = new CompletableFuture<>();

private final Optional<RestClientAuth> auth;

private ClientHandler(final Optional<RestClientAuth> auth) {
this.auth = auth;
}

CompletableFuture<JsonResponse> getJsonFuture() {
return jsonFuture;
}

@Override
protected void channelRead0(ChannelHandlerContext ctx, Object msg) {
if (msg instanceof HttpResponse
&& ((HttpResponse) msg).status().equals(REQUEST_ENTITY_TOO_LARGE)) {
jsonFuture.completeExceptionally(
new RestClientException(
String.format(
REQUEST_ENTITY_TOO_LARGE + ". Try to raise [%s]",
RestOptions.CLIENT_MAX_CONTENT_LENGTH.key()),
((HttpResponse) msg).status()));
} else if (msg instanceof FullHttpResponse) {
readRawResponse((FullHttpResponse) msg);
} else {
LOG.error(
"Implementation error: Received a response that wasn't a FullHttpResponse.");
if (msg instanceof HttpResponse) {
boolean implementationError = true;
HttpResponseStatus status = INTERNAL_SERVER_ERROR;
if (msg instanceof HttpResponse) {
implementationError = false;
final HttpResponse response = (HttpResponse) msg;
status = response.status();
if (status.equals(REQUEST_ENTITY_TOO_LARGE)) {
jsonFuture.completeExceptionally(
new RestClientException(
"Implementation error: Received a response that wasn't a FullHttpResponse.",
((HttpResponse) msg).getStatus()));
String.format(
REQUEST_ENTITY_TOO_LARGE + ". Try to raise [%s]",
RestOptions.CLIENT_MAX_CONTENT_LENGTH.key()),
status));
} else if (status.equals(UNAUTHORIZED)
|| status.equals(PROXY_AUTHENTICATION_REQUIRED)) {
final Exception ex;
if (auth.map(a -> a.isRestartable(response)).orElse(false)) {
ex =
new RestartableException(
"Restart to response authentication challenge", auth);
} else {
ex = new RestClientException("Failed to authenticate", status);
}
jsonFuture.completeExceptionally(ex);
} else if (msg instanceof FullHttpResponse) {
readRawResponse((FullHttpResponse) msg);
} else {
jsonFuture.completeExceptionally(
new RestClientException(
"Implementation error: Received a response that wasn't a FullHttpResponse.",
HttpResponseStatus.INTERNAL_SERVER_ERROR));
implementationError = true;
}
}

if (implementationError) {
LOG.error(
"Implementation error: Received a response that wasn't a FullHttpResponse.");
jsonFuture.completeExceptionally(
new RestClientException(
"Implementation error: Received a response that wasn't a FullHttpResponse.",
status));
}
ctx.close();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,15 @@
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.runtime.io.network.netty.SSLHandlerFactory;
import org.apache.flink.runtime.net.SSLUtils;
import org.apache.flink.runtime.rest.auth.RestClientAuth;
import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.Preconditions;

import javax.annotation.Nullable;
import javax.net.ssl.SSLEngine;

import java.util.Optional;

import static org.apache.flink.util.Preconditions.checkArgument;

/** A configuration object for {@link RestClient}s. */
Expand All @@ -41,11 +44,14 @@ public final class RestClientConfiguration {

private final int maxContentLength;

private final Optional<RestClientAuth> auth;

private RestClientConfiguration(
@Nullable final SSLHandlerFactory sslHandlerFactory,
final long connectionTimeout,
final long idlenessTimeout,
final int maxContentLength) {
final int maxContentLength,
@Nullable final Optional<RestClientAuth> auth) {
checkArgument(
maxContentLength > 0,
"maxContentLength must be positive, was: %s",
Expand All @@ -54,6 +60,7 @@ private RestClientConfiguration(
this.connectionTimeout = connectionTimeout;
this.idlenessTimeout = idlenessTimeout;
this.maxContentLength = maxContentLength;
this.auth = auth;
}

/**
Expand Down Expand Up @@ -85,6 +92,10 @@ public int getMaxContentLength() {
return maxContentLength;
}

public Optional<RestClientAuth> getAuth() {
return auth.map(RestClientAuth::copy);
}

/**
* Creates and returns a new {@link RestClientConfiguration} from the given {@link
* Configuration}.
Expand Down Expand Up @@ -114,9 +125,11 @@ public static RestClientConfiguration fromConfiguration(Configuration config)

final long idlenessTimeout = config.getLong(RestOptions.IDLENESS_TIMEOUT);

int maxContentLength = config.getInteger(RestOptions.CLIENT_MAX_CONTENT_LENGTH);
final int maxContentLength = config.getInteger(RestOptions.CLIENT_MAX_CONTENT_LENGTH);

final Optional<RestClientAuth> auth = RestClientAuth.fromConfiguration(config);

return new RestClientConfiguration(
sslHandlerFactory, connectionTimeout, idlenessTimeout, maxContentLength);
sslHandlerFactory, connectionTimeout, idlenessTimeout, maxContentLength, auth);
}
}
Loading