You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by rh...@apache.org on 2020/05/24 14:10:10 UTC
[kafka] branch trunk updated: KAFKA-9767: Add logging to basic auth
rest extension (#8357)
This is an automated email from the ASF dual-hosted git repository.
rhauch pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 85ed123 KAFKA-9767: Add logging to basic auth rest extension (#8357)
85ed123 is described below
commit 85ed123ac65d7459df5049b75178cf7bcc9c9e79
Author: Chris Egerton <ch...@confluent.io>
AuthorDate: Sun May 24 07:09:40 2020 -0700
KAFKA-9767: Add logging to basic auth rest extension (#8357)
Add logging to basic auth rest extension.
Author: Chris Egerton <ch...@confluent.io>
Reviewers: Magesh Nandakumar <ma...@gmail.com>, Randall Hauch <rh...@gmail.com>
---
.../extension/BasicAuthSecurityRestExtension.java | 6 ++
.../basic/auth/extension/JaasBasicAuthFilter.java | 94 ++++++++++++++++------
.../auth/extension/PropertyFileLoginModule.java | 41 +++++++++-
.../auth/extension/JaasBasicAuthFilterTest.java | 25 +++++-
4 files changed, 136 insertions(+), 30 deletions(-)
diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java
index 4169e9e..21e13c2 100644
--- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java
+++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/BasicAuthSecurityRestExtension.java
@@ -20,6 +20,8 @@ package org.apache.kafka.connect.rest.basic.auth.extension;
import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.connect.rest.ConnectRestExtension;
import org.apache.kafka.connect.rest.ConnectRestExtensionContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Map;
@@ -57,9 +59,13 @@ import java.util.Map;
*/
public class BasicAuthSecurityRestExtension implements ConnectRestExtension {
+ private static final Logger log = LoggerFactory.getLogger(BasicAuthSecurityRestExtension.class);
+
@Override
public void register(ConnectRestExtensionContext restPluginContext) {
+ log.trace("Registering JAAS basic auth filter");
restPluginContext.configurable().register(JaasBasicAuthFilter.class);
+ log.trace("Finished registering JAAS basic auth filter");
}
@Override
diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java
index d5b15c6..8326239 100644
--- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java
+++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java
@@ -17,9 +17,14 @@
package org.apache.kafka.connect.rest.basic.auth.extension;
+import java.util.ArrayList;
+import java.util.List;
import java.util.regex.Pattern;
import javax.ws.rs.HttpMethod;
import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@@ -37,19 +42,29 @@ import javax.ws.rs.container.ContainerRequestFilter;
import javax.ws.rs.core.Response;
public class JaasBasicAuthFilter implements ContainerRequestFilter {
+
+ private static final Logger log = LoggerFactory.getLogger(JaasBasicAuthFilter.class);
+ private static final Pattern TASK_REQUEST_PATTERN = Pattern.compile("/?connectors/([^/]+)/tasks/?");
private static final String CONNECT_LOGIN_MODULE = "KafkaConnect";
+
static final String AUTHORIZATION = "Authorization";
- private static final Pattern TASK_REQUEST_PATTERN = Pattern.compile("/?connectors/([^/]+)/tasks/?");
+
@Override
public void filter(ContainerRequestContext requestContext) throws IOException {
+ if (isInternalTaskConfigRequest(requestContext)) {
+ log.trace("Skipping authentication for internal request");
+ return;
+ }
+
try {
- if (!(requestContext.getMethod().equals(HttpMethod.POST) && TASK_REQUEST_PATTERN.matcher(requestContext.getUriInfo().getPath()).matches())) {
- LoginContext loginContext =
- new LoginContext(CONNECT_LOGIN_MODULE, new BasicAuthCallBackHandler(
- requestContext.getHeaderString(AUTHORIZATION)));
- loginContext.login();
- }
+ log.debug("Authenticating request");
+ LoginContext loginContext =
+ new LoginContext(CONNECT_LOGIN_MODULE, new BasicAuthCallBackHandler(
+ requestContext.getHeaderString(AUTHORIZATION)));
+ loginContext.login();
} catch (LoginException | ConfigException e) {
+ // Log at debug here in order to avoid polluting log files whenever someone mistypes their credentials
+ log.debug("Request failed authentication", e);
requestContext.abortWith(
Response.status(Response.Status.UNAUTHORIZED)
.entity("User cannot access the resource.")
@@ -57,6 +72,11 @@ public class JaasBasicAuthFilter implements ContainerRequestFilter {
}
}
+ private static boolean isInternalTaskConfigRequest(ContainerRequestContext requestContext) {
+ return requestContext.getMethod().equals(HttpMethod.POST)
+ && TASK_REQUEST_PATTERN.matcher(requestContext.getUriInfo().getPath()).matches();
+ }
+
public static class BasicAuthCallBackHandler implements CallbackHandler {
@@ -67,36 +87,60 @@ public class JaasBasicAuthFilter implements ContainerRequestFilter {
private String password;
public BasicAuthCallBackHandler(String credentials) {
- if (credentials != null) {
- int space = credentials.indexOf(SPACE);
- if (space > 0) {
- String method = credentials.substring(0, space);
- if (BASIC.equalsIgnoreCase(method)) {
- credentials = credentials.substring(space + 1);
- credentials = new String(Base64.getDecoder().decode(credentials),
- StandardCharsets.UTF_8);
- int i = credentials.indexOf(COLON);
- if (i > 0) {
- username = credentials.substring(0, i);
- password = credentials.substring(i + 1);
- }
- }
- }
+ if (credentials == null) {
+ log.trace("No credentials were provided with the request");
+ return;
}
+
+ int space = credentials.indexOf(SPACE);
+ if (space <= 0) {
+ log.trace("Request credentials were malformed; no space present in value for authorization header");
+ return;
+ }
+
+ String method = credentials.substring(0, space);
+ if (!BASIC.equalsIgnoreCase(method)) {
+ log.trace("Request credentials used {} authentication, but only {} supported; ignoring", method, BASIC);
+ return;
+ }
+
+ credentials = credentials.substring(space + 1);
+ credentials = new String(Base64.getDecoder().decode(credentials),
+ StandardCharsets.UTF_8);
+ int i = credentials.indexOf(COLON);
+ if (i <= 0) {
+ log.trace("Request credentials were malformed; no colon present between username and password");
+ return;
+ }
+
+ username = credentials.substring(0, i);
+ password = credentials.substring(i + 1);
}
@Override
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+ List<Callback> unsupportedCallbacks = new ArrayList<>();
for (Callback callback : callbacks) {
if (callback instanceof NameCallback) {
((NameCallback) callback).setName(username);
} else if (callback instanceof PasswordCallback) {
- ((PasswordCallback) callback).setPassword(password.toCharArray());
+ ((PasswordCallback) callback).setPassword(password != null
+ ? password.toCharArray()
+ : null
+ );
} else {
- throw new UnsupportedCallbackException(callback, "Supports only NameCallback "
- + "and PasswordCallback");
+ unsupportedCallbacks.add(callback);
}
}
+ if (!unsupportedCallbacks.isEmpty())
+ throw new ConnectException(String.format(
+ "Unsupported callbacks %s; request authentication will fail. "
+ + "This indicates the Connect worker was configured with a JAAS "
+ + "LoginModule that is incompatible with the %s, and will need to be "
+ + "corrected and restarted.",
+ unsupportedCallbacks,
+ BasicAuthSecurityRestExtension.class.getSimpleName()
+ ));
}
}
}
diff --git a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java
index 8a013330..8a26dc3 100644
--- a/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java
+++ b/connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/PropertyFileLoginModule.java
@@ -62,17 +62,27 @@ public class PropertyFileLoginModule implements LoginModule {
if (fileName == null || fileName.trim().isEmpty()) {
throw new ConfigException("Property Credentials file must be specified");
}
+
if (!credentialPropertiesMap.containsKey(fileName)) {
+ log.trace("Opening credential properties file '{}'", fileName);
Properties credentialProperties = new Properties();
try {
try (InputStream inputStream = Files.newInputStream(Paths.get(fileName))) {
+ log.trace("Parsing credential properties file '{}'", fileName);
credentialProperties.load(inputStream);
}
credentialPropertiesMap.putIfAbsent(fileName, credentialProperties);
+ if (credentialProperties.isEmpty())
+ log.warn("Credential properties file '{}' is empty; all requests will be permitted",
+ fileName);
} catch (IOException e) {
log.error("Error loading credentials file ", e);
throw new ConfigException("Error loading Property Credentials file");
}
+ } else {
+ log.trace(
+ "Credential properties file '{}' has already been opened and parsed; will read from cached, in-memory store",
+ fileName);
}
}
@@ -80,8 +90,10 @@ public class PropertyFileLoginModule implements LoginModule {
public boolean login() throws LoginException {
Callback[] callbacks = configureCallbacks();
try {
+ log.trace("Authenticating user; invoking JAAS login callbacks");
callbackHandler.handle(callbacks);
} catch (Exception e) {
+ log.warn("Authentication failed while invoking JAAS login callbacks", e);
throw new LoginException(e.getMessage());
}
@@ -89,8 +101,32 @@ public class PropertyFileLoginModule implements LoginModule {
char[] passwordChars = ((PasswordCallback) callbacks[1]).getPassword();
String password = passwordChars != null ? new String(passwordChars) : null;
Properties credentialProperties = credentialPropertiesMap.get(fileName);
- authenticated = credentialProperties.isEmpty() ||
- (password != null && password.equals(credentialProperties.get(username)));
+
+ if (credentialProperties.isEmpty()) {
+ log.trace("Not validating credentials for user '{}' as credential properties file '{}' is empty",
+ username,
+ fileName);
+ authenticated = true;
+ } else if (username == null) {
+ log.trace("No credentials were provided or the provided credentials were malformed");
+ authenticated = false;
+ } else if (password != null && password.equals(credentialProperties.get(username))) {
+ log.trace("Credentials provided for user '{}' match those present in the credential properties file '{}'",
+ username,
+ fileName);
+ authenticated = true;
+ } else if (!credentialProperties.containsKey(username)) {
+ log.trace("User '{}' is not present in the credential properties file '{}'",
+ username,
+ fileName);
+ authenticated = false;
+ } else {
+ log.trace("Credentials provided for user '{}' do not match those present in the credential properties file '{}'",
+ username,
+ fileName);
+ authenticated = false;
+ }
+
return authenticated;
}
@@ -110,7 +146,6 @@ public class PropertyFileLoginModule implements LoginModule {
}
private Callback[] configureCallbacks() {
-
Callback[] callbacks = new Callback[2];
callbacks[0] = new NameCallback("Enter user name");
callbacks[1] = new PasswordCallback("Enter password", false);
diff --git a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java
index fe5f8b9..0c39c5f 100644
--- a/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java
+++ b/connect/basic-auth-extension/src/test/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilterTest.java
@@ -17,9 +17,13 @@
package org.apache.kafka.connect.rest.basic.auth.extension;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.ChoiceCallback;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.core.UriInfo;
import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.connect.errors.ConnectException;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Before;
@@ -169,11 +173,28 @@ public class JaasBasicAuthFilterTest {
EasyMock.verify(requestContext);
}
+ @Test(expected = ConnectException.class)
+ public void testUnsupportedCallback() throws Exception {
+ String authHeader = authHeader("basic", "user", "pwd");
+ CallbackHandler callbackHandler = new JaasBasicAuthFilter.BasicAuthCallBackHandler(authHeader);
+ Callback unsupportedCallback = new ChoiceCallback(
+ "You take the blue pill... the story ends, you wake up in your bed and believe whatever you want to believe. "
+ + "You take the red pill... you stay in Wonderland, and I show you how deep the rabbit hole goes.",
+ new String[] {"blue pill", "red pill"},
+ 1,
+ true
+ );
+ callbackHandler.handle(new Callback[] {unsupportedCallback});
+ }
+
+ private String authHeader(String authorization, String username, String password) {
+ return authorization + " " + Base64.getEncoder().encodeToString((username + ":" + password).getBytes());
+ }
+
private void setMock(String authorization, String username, String password, boolean exceptionCase) {
EasyMock.expect(requestContext.getMethod()).andReturn(HttpMethod.GET);
- String authHeader = authorization + " " + Base64.getEncoder().encodeToString((username + ":" + password).getBytes());
EasyMock.expect(requestContext.getHeaderString(JaasBasicAuthFilter.AUTHORIZATION))
- .andReturn(authHeader);
+ .andReturn(authHeader(authorization, username, password));
if (exceptionCase) {
requestContext.abortWith(EasyMock.anyObject(Response.class));
EasyMock.expectLastCall();