You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 2 Next »

This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.

Status

Current state: "Under Discussion"

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Kafka currently supports SASL authentication using SASL/PLAIN mechanism and KIP-84 addresses the addition of SASL/SCRAM. Credential verification in SASL/PLAIN servers is currently based on hard-coded credentials in JAAS configuration similar to Digest-MD5 configuration in Zookeeper. This is useful as a sample, but not suitable for production use since clear passwords are stored on disk. KIP-84 proposes to add SCRAM mechanism with Zookeeper as the password store. In production installations where Zookeeper is not secure, an alternative password store may be required.

With the current Kafka implementation, the entire SaslServer implementation needs to be replaced to enable new credential providers. It will be useful to add an extension point for SASL that enables just the credential providers to be replaced with a custom implementation. This should be done in a consistent way for all SASL mechanisms.

This KIP proposes to enable customization of SASL server and clients using configurable callback handlers. Configurable callback handlers for SASL/PLAIN and SASL/SCRAM will enable credential providers to be replaced in a simple and consistent way. In addition to this, configurable callback handlers for both server and clients make it easier to configure new SASL mechanisms that are not implemented in Kafka.

Public Interfaces

Configuration property

A new configuration property sasl.callback.handlers will be added to enable new callback handlers to be specified for brokers and clients. This will be a list of classes that implement the org.apache.kafka.common.security.auth.AuthCallbackHandler interface. A different handler may be provided for each enabled mechanism.

Callback Handler

The callback handler interface AuthCallbackHandler will extend the standard javax.security.auth.callback.CallbackHandler interface, enabling the handler to be passed directly to SaslServer/SaslClient implementations. The callback handler configured for a mechanism must include the callbacks as described below:

  1. If using a SaslServer/SaslClient implementation from the JRE, the callbacks required for the mechanism are described in the Java SASL reference.
  2. When using the SaslServer/SaslClient implementation included in Kafka (PLAIN or SCRAM), the callback defined below for the required SASL mechanism must be handled.
  3. Applications using custom implementations of SaslServer/SaslClient may define their own callbacks.

Callback handlers which require additional options at runtime (eg. URL of a credential server) may include arguments in the JAAS configuration using the sasl.jaas.config property (KIP-85). This is similar to the way keytab location is configured for GSSAPI. Client callback handlers can retrieve Subject using Subject.getSubject(AccessController.getContext()) to obtain credentials populated by the login module.

org.apache.kafka.common.security.auth.AuthCallbackHandler
package org.apache.kafka.common.security.auth;
import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.network.Mode;
import java.util.Collection;
import javax.security.auth.callback.CallbackHandler;

public interface AuthCallbackHandler extends Configurable, CallbackHandler {
    /**
     * Returns the connection mode supported by this callback handler
     */
    Mode mode();
    /**
     * Returns the SASL mechanisms supported by this callback handler
     */
    Collection<String> saslMechanisms();
    /**
     * Closes this instance.
     */
    void close();
}

 

SASL/PLAIN Callback

org.apache.kafka.common.security.plain.PlainCredentialCallback
package org.apache.kafka.common.security.plain;
import javax.security.auth.callback.Callback;

public class PlainCredentialCallback implements Callback {
    private final String username;
    private final String password;
    private boolean authenticated;
    public PlainCredentialCallback(String username, String password) {
        this.username = username;
        this.password = password;
    }
    public String username() {
        return username;
    }
    public String password() {
        return password;
    }
    public boolean authenticated() {
        return this.authenticated;
    }
    public void authenticated(boolean success) {
        this.authenticated = success;
    }
}

SASL/SCRAM Callback

org.apache.kafka.common.security.scram.ScramCredentialCallback
package org.apache.kafka.common.security.scram;
import javax.security.auth.callback.Callback;
public class ScramCredentialCallback implements Callback {
    private final String username;
    private String salt;
    private byte[] serverKey;
    private byte[] storedKey;
    private int iterationCount;
    public ScramCredentialCallback(String username) {
        this.username = username;
    }
    public String username() {
        return username;
    }
    public String salt() {
        return salt;
    }
    public void salt(String salt) {
        this.salt = salt;
    }
    public byte[] serverKey() {
        return serverKey;
    }
    public void serverKey(byte[] serverKey) {
        this.serverKey = serverKey;
    }
    public byte[] storedKey() {
        return storedKey;
    }
    public void storedKey(byte[] storedKey) {
        this.storedKey = storedKey;
    }
    public int iterationCount() {
        return iterationCount;
    }
    public void iterationCount(int iterationCount) {
        this.iterationCount = iterationCount;
    }
}

 

Proposed Changes

ChannelBuilder  will create an instance of each configured callback handler using the default constructor. For mechanisms without a callback handler override, the existing default callback handlers (SaslServerCallbackHandler/SaslClientCallbackHandler) will be created once for each mechanism in ChannelBuilder, instead of per-connection. This enables callback handlers using external authentication servers to cache credentials or reuse connections if required. SaslClientCallbackHandler will be modified to obtain Subject using Subject.getSubject(AccessController.getContext()) to avoid the current per-connection state.

Scenarios 

Add new credential provider for SASL/PLAIN using the SaslServer implementation for PLAIN included in Kafka 

 Define a new class that implements AuthCallbackHandler  which handles PlainCredentialCallback and add the class to the broker's sasl.callback.handlers property. A single instance of this callback handler will be created for the broker. The configured callback handler is responsible for validating the password provided by clients. SASL/SCRAM can be configured in a similar way with a callback handler that implements ScramCredentialCallback.

Use a custom SaslServer implementation for SCRAM

If a custom SaslServer implementation is used instead of the one included in Kafka, the custom implementation may require a different set of callbacks. A callback handler for these callbacks may be specified in sasl.callback.handlers.

Configure a new mechanism not included in Kafka using custom SaslServer/SaslClient

A handler that handles any callbacks required for these server/client implementations may be specified in sasl.callback.handlers for brokers and clients

Configure a new mechanism using the implementation provided by the JRE

Callbacks defined for the mechanism in the Java implementation must be handled by custom callback handlers if the behaviour differs from the default callbacks in Kafka.


Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?

None

  • If we are changing behavior how will we phase out the older behavior?

Existing behaviour will be retained as default

Test Plan

Existing integration and system tests will test the default behaviour. Additional unit and integration tests will be added to test the new configuration:

  1. Test that PLAIN credential provider can be replaced
  2. Test that SCRAM password store can be replaced
  3. Test that new mechanisms not included in Kafka can be run with custom callback handlers

Rejected Alternatives

Define a new credential provider interface instead of using CallbackHandler

The format of credentials required for each mechanism is different. For SASL/PLAIN, the proposed callback handler can be used with external authentication servers which validate passwords without allowing the broker to retrieve password for a user. For SASL/SCRAM, the hashed, salted credentials required for the mechanism are provided to the broker. Different credential providers can be defined to capture these differences, which may make the interface slightly simpler in these two cases compared to callback handlers. But the use of standard Java CallbackHandler interface is more flexible and future-proof since it is the interface used by SaslServer/SaslClient implementations to obtain application-specific data.


  • No labels