This is an automated email from the ASF dual-hosted git repository. ijokarumawak pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/master by this push: new 4166392 NIFI-6174: Add ClientAuth property to ListenBeats 4166392 is described below commit 41663929a4727592972e6be04b3c516a752e760e Author: DimDroll <dimdr...@gmail.com> AuthorDate: Wed Jun 19 15:59:52 2019 +0300 NIFI-6174: Add ClientAuth property to ListenBeats This closes #3539. Signed-off-by: Koji Kawamura <ijokaruma...@apache.org> --- .../apache/nifi/processors/beats/ListenBeats.java | 30 +++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java index af5d1b4..6a4a568 100644 --- a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java +++ b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java @@ -28,6 +28,8 @@ import java.util.Map; import java.util.concurrent.BlockingQueue; import javax.net.ssl.SSLContext; +import org.apache.nifi.security.util.SslContextFactory; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.WritesAttribute; @@ -85,11 +87,21 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor<BeatsEvent .identifiesControllerService(RestrictedSSLContextService.class) .build(); + public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder() + .name("Client Auth") + .displayName("Client Auth") + .description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.") + .required(false) + .allowableValues(RestrictedSSLContextService.ClientAuth.values()) + .defaultValue(RestrictedSSLContextService.ClientAuth.REQUIRED.name()) + .build(); + @Override protected List<PropertyDescriptor> getAdditionalProperties() { return Arrays.asList( MAX_CONNECTIONS, - SSL_CONTEXT_SERVICE + SSL_CONTEXT_SERVICE, + CLIENT_AUTH ); } @@ -105,6 +117,14 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor<BeatsEvent .valid(false).subject(SSL_CONTEXT_SERVICE.getName()).build()); } + // Validate CLIENT_AUTH + final String clientAuth = validationContext.getProperty(CLIENT_AUTH).getValue(); + if (sslContextService != null && StringUtils.isBlank(clientAuth)) { + results.add(new ValidationResult.Builder() + .explanation("Client Auth must be provided when using TLS/SSL") + .valid(false).subject("Client Auth").build()); + } + return results; } @@ -133,14 +153,18 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor<BeatsEvent // if an SSLContextService was provided then create an SSLContext to pass down to the dispatcher SSLContext sslContext = null; + SslContextFactory.ClientAuth clientAuth = null; final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); if (sslContextService != null) { - sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED); + final String clientAuthValue = context.getProperty(CLIENT_AUTH).getValue(); + sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.valueOf(clientAuthValue)); + clientAuth = SslContextFactory.ClientAuth.valueOf(clientAuthValue); + } // if we decide to support SSL then get the context and pass it in here return new SocketChannelDispatcher<>(eventFactory, handlerFactory, bufferPool, events, - getLogger(), maxConnections, sslContext, charSet); + getLogger(), maxConnections, sslContext, clientAuth, charSet); }