Skip to content

Comments

KAFKA-6750: Add listener name to authentication context (KIP-282)#4829

Merged
ijuma merged 6 commits intoapache:trunkfrom
mimaison:listener-name
Jun 5, 2018
Merged

KAFKA-6750: Add listener name to authentication context (KIP-282)#4829
ijuma merged 6 commits intoapache:trunkfrom
mimaison:listener-name

Conversation

@mimaison
Copy link
Member

@mimaison mimaison commented Apr 5, 2018

Co-authored-by: Edoardo Comar ecomar@uk.ibm.com
Co-authored-by: Mickael Maison mickael.maison@gmail.com

This KIP adds the String value of the ListenerName used for the connection to the AuthenticationContext.

This allows PrincipalBuilders to retrieve this value and build different Principal based on the source of the connection. This is especially interesting in deployments where inter-broker traffic is on a different network/listener than user traffic or when the same protocol is used by several listeners.

The change in itself is mostly "plumbing" as the listener name needs to be passed from ChannelBuilders all the way down to all classes implementing AuthenticationContext.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@mimaison mimaison force-pushed the listener-name branch 2 times, most recently from cabe8e1 to ed2d367 Compare April 18, 2018 10:19
…text

Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
@ijuma ijuma self-requested a review May 21, 2018 18:59
@ijuma ijuma added this to the 2.0.0 milestone May 21, 2018
@ijuma
Copy link
Member

ijuma commented May 21, 2018

Thanks for the PR. FYI, the text in italics in the PR is meant to be replaced with your own message. ;)

@ijuma ijuma self-assigned this May 21, 2018
@mimaison
Copy link
Member Author

Thanks @ijuma I updated the description accordingly 😄

import org.junit.Assert._
import org.junit.{Before, Test}

class ListenerAndPrincipalBuilderTest extends BaseRequestTest {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need a new test for this? These tests are expensive since they involve starting several brokers. Ideally, we'd update one of the existing tests to also check the listener name.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We had a look at other tests extending BaseRequestTest and this does not really fit anywhere. We tried merging it into KafkaMetricReporterExceptionHandlingTest (one test checking both logics) but it really looked awful. As each test needs its own setup, the overall logic made little sense even with comments.

Yes the test takes 5 secs but we think it's clearer to keep it separated.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ijuma As the deadline is today, can you have a look?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about PlaintextEndToEndAuthorizationTest?

@ijuma
Copy link
Member

ijuma commented May 29, 2018

This is a small feature, so we have an extra week.

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the PR, left a few comments.

public class PlaintextChannelBuilder implements ChannelBuilder {
private static final Logger log = LoggerFactory.getLogger(PlaintextChannelBuilder.class);
private Map<String, ?> configs;
private ListenerName listenerName;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should be final.

public class ChannelBuildersTest {

@Test
@SuppressWarnings("deprecation")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why did we remove this?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test does not use any deprecated classes, so it's unecessary

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, OldPrincipalBuilder extends the deprecated class but it's not deprecated itself.

} catch (KafkaException e) {
// Expected exception
} finally {
channelBuilder.close();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use try with resources?

import org.junit.Assert._
import org.junit.{Before, Test}

class ListenerAndPrincipalBuilderTest extends BaseRequestTest {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about PlaintextEndToEndAuthorizationTest?

InetAddress clientAddress = transportLayer.socketChannel().socket().getInetAddress();
return principalBuilder.build(new PlaintextAuthenticationContext(clientAddress));
// listenerName should only be null in Client mode
String listenerNameStr = listenerName != null ? listenerName.value() : null;
Copy link
Member

@ijuma ijuma Jun 2, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems a bit brittle. Maybe the field should be Optional<ListenerName>?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should never be calling principal() in client mode.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should probably not have the null check then, better fail fast. Same for the other case.

/**
* Name of the listener used for the connection
*/
String listenerName();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be null? Maybe we should return Optional<String>. We would need to update the KIP too in that case.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AuthenticationContext is only used on the server-side, so listener name should never be null.

@mimaison
Copy link
Member Author

mimaison commented Jun 4, 2018

Regarding listenerName being null. This can only happen on the Client side. The Client side only uses the default principal builder (even though the Client side is not using the Principal) and that does not use the new listener field.

The listenerName field will always be set on the server side where a custom PrincipalBuilder can try to access this field.

We don't have to check for null in SaslChannelBuilder as it has a Client and a Server side Authenticator so we can only pass the listener when on server.

In my opinion, having an Optional when it's always going to be set when needed is not the ideal solution. Maybe we can make this clearer in comments. What do you think ?

@ijuma
Copy link
Member

ijuma commented Jun 4, 2018

@mimaison, AuthenticationContext is used in the client and the server. The fact that the listenerName is only used in the broker is even more reason to make it Optional. We can then document that it's always defined in the broker and never defined in the client. The fact that we currently have multiple places in the code with the following line is worrying:

// listenerName should only be null in Client mode
+            String listenerNameStr = listenerName != null ? listenerName.value() : null;

Another option would be to have a ServerAuthenticationContext subclass. But not sure if that's easy to add.

Am I missing something? Also cc @rajinisivaram in case she has some thoughts.

@rajinisivaram
Copy link
Contributor

@ijuma There are a few internal methods where listenerName is passed through which are used both by client and server and these need null checks (could even be IllegalStateException, but maybe easier to leave as null checks for now). But the public interface AuthenticationContext is only for use on the server-side to build the principal. So I think it makes sense to leave listenerName as a String rather than make it Optional.

@ijuma
Copy link
Member

ijuma commented Jun 5, 2018

Synced with @rajinisivaram offline and AuthenticationContext is callable from both server and client, but we only access KafkaChannel.principal() on the server. It would be good to improve the code so that KafkaChannel.principal() works in both cases (i.e. if the principal builder is only meant to be called on the server, then the client path should return the principal associated with the connection without going through the principal builder (this is what happens in the SaslClientAuthenticator case as @mimaison said). In any case, for another PR. :)

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A few minor comments and then I think this is ready to be merged.

public class PlaintextChannelBuilder implements ChannelBuilder {
private static final Logger log = LoggerFactory.getLogger(PlaintextChannelBuilder.class);
private Map<String, ?> configs;
private final ListenerName listenerName;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

static fields first, then final, then mutable.


/**
* Constructs a plaintext channel builder. ListenerName is provided only
* for server channel builder and will be null for client channel builder.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's actually a bit more complicated. It's non-null whenever it's instantiated in the broker and null otherwise. In the broker case, we create both server and client channel builders, but it's set in both (for sending and receiving requests respectively).

InetAddress clientAddress = transportLayer.socketChannel().socket().getInetAddress();
return principalBuilder.build(new PlaintextAuthenticationContext(clientAddress));
// listenerName should only be null in Client mode
String listenerNameStr = listenerName != null ? listenerName.value() : null;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should probably not have the null check then, better fail fast. Same for the other case.

* An object representing contextual information from the authentication session. See
* {@link SaslAuthenticationContext} and {@link SslAuthenticationContext}.
* {@link PlaintextAuthenticationContext}, {@link SaslAuthenticationContext}
* and {@link SslAuthenticationContext}.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we should mention that this class is only used in the broker.

package org.apache.kafka.common.security.auth;

import javax.net.ssl.SSLSession;

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this intentional?

transportLayer, oldPrincipalBuilder, null);

KafkaPrincipal principal = builder.build(new PlaintextAuthenticationContext(InetAddress.getLocalHost()));
KafkaPrincipal principal = builder.build(new PlaintextAuthenticationContext(InetAddress.getLocalHost(), SecurityProtocol.PLAINTEXT.name()));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A few long lines in this class. The goal is that they should fit within the GitHub window.

}

private def sendRecords(numRecords: Int, tp: TopicPartition) {
protected def sendRecords(numRecords: Int, tp: TopicPartition) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

protected final?

// which we derive corresponding principals
object PlaintextEndToEndAuthorizationTest {
var clientListenerName: String = ""
var serverListenerName: String = ""
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

private and volatile? Also, it seems like initialising them as None would be better.

fail("Should have thrown a TopicAuthorizationException")
} catch {
case tae: TopicAuthorizationException => //expected
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use intercept[TopicAuthorizationException](sendRecords(1, tp)

@ijuma ijuma changed the title KAFKA-6750 - KIP-282: Add the listener name to the authentication con… KAFKA-6750: Add the listener name to the authentication context (KIP-282) Jun 5, 2018
@ijuma ijuma changed the title KAFKA-6750: Add the listener name to the authentication context (KIP-282) KAFKA-6750: Add listener name to authentication context (KIP-282) Jun 5, 2018
@mimaison
Copy link
Member Author

mimaison commented Jun 5, 2018

@ijuma Thanks for the feedback! I've addressed all your comments.

intercept[TopicAuthorizationException](sendRecords(1, tp))

assertEquals("CLIENT", PlaintextEndToEndAuthorizationTest.clientListenerName.get)
assertEquals("SERVER", PlaintextEndToEndAuthorizationTest.serverListenerName.get)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The right way to do this is assertEquals(Some("SERVER"), PlaintextEndToEndAuthorizationTest.serverListenerName) so that one gets a reasonable error if the Option is None.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point, fixed

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks.

@ijuma ijuma merged commit 8a166f8 into apache:trunk Jun 5, 2018
ijuma added a commit to edoardocomar/kafka that referenced this pull request Jun 6, 2018
…grained-acl-create-topics

* apache-github/trunk:
  KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097)
  MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040)
  KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE
  MINOR: docs should point to latest version (apache#5132)
  KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298)
  [KAFKA-6730] Simplify State Store Recovery (apache#5013)
  MINOR: Rename package `internal` to `internals` for consistency (apache#5137)
  KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801)
  MINOR: Add missing configs for resilience settings
  MINOR: Add regression tests for KTable mapValues and filter (apache#5134)
  KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829)
  KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956)
  KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128)
  KAFKA-6813: return to double-counting for count topology names (apache#5075)
  KAFKA-5919; Adding checks on "version" field for tools using it
  MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
ijuma added a commit to big-andy-coates/kafka that referenced this pull request Jun 6, 2018
…refix

* apache-github/trunk:
  KAFKA-6726: Fine Grained ACL for CreateTopics (KIP-277) (apache#4795)
  KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097)
  MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040)
  KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE
  MINOR: docs should point to latest version (apache#5132)
  KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298)
  [KAFKA-6730] Simplify State Store Recovery (apache#5013)
  MINOR: Rename package `internal` to `internals` for consistency (apache#5137)
  KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801)
  MINOR: Add missing configs for resilience settings
  MINOR: Add regression tests for KTable mapValues and filter (apache#5134)
  KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829)
  KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956)
  KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128)
  KAFKA-6813: return to double-counting for count topology names (apache#5075)
  KAFKA-5919; Adding checks on "version" field for tools using it
  MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
ying-zheng pushed a commit to ying-zheng/kafka that referenced this pull request Jul 6, 2018
…ache#4829)

PrincipalBuilder implementations can now take the listener into account
when creating the Principal. This is especially interesting in deployments
where inter-broker traffic is on a different listener than client traffic or
when the same protocol is used by multiple listeners.

The change in itself is mostly "plumbing" as the listener name needs to be
passed from ChannelBuilders all the way down to all classes implementing
AuthenticationContext.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>

Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
@mimaison mimaison deleted the listener-name branch May 15, 2020 23:05
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants