Skip to content

Commit

Permalink
FLUME-3278 Handling -D keystore parameters in Kafka components
Browse files Browse the repository at this point in the history
Kafka client does not handle -D keystore parameters directly so Flume has to
pass them explicitly in Kafka properties (like ssl.keystore.location, etc).
Also using the same method for the truststore (in order to handle
keystore/truststore in the same way).

This closes #231

Reviewers: Denes Arvay

(Peter Turcsanyi via Ferenc Szabo)
  • Loading branch information
turcsanyip authored and szaboferee committed Oct 29, 2018
1 parent 965e132 commit 493b53b
Show file tree
Hide file tree
Showing 11 changed files with 329 additions and 0 deletions.
4 changes: 4 additions & 0 deletions flume-ng-channels/flume-kafka-channel/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,10 @@ limitations under the License.
<groupId>org.apache.flume</groupId>
<artifactId>flume-ng-sdk</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka-test</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.flume.conf.LogPrivacyUtil;
import org.apache.flume.event.EventBuilder;
import org.apache.flume.instrumentation.kafka.KafkaChannelCounter;
import org.apache.flume.shared.kafka.KafkaSSLUtil;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
Expand Down Expand Up @@ -268,6 +269,8 @@ private void setProducerProps(Context ctx, String bootStrapServers) {
// Defaults overridden based on config
producerProps.putAll(ctx.getSubProperties(KAFKA_PRODUCER_PREFIX));
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);

KafkaSSLUtil.addGlobalSSLParameters(producerProps);
}

protected Properties getProducerProps() {
Expand All @@ -285,6 +288,8 @@ private void setConsumerProps(Context ctx, String bootStrapServers) {
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);

KafkaSSLUtil.addGlobalSSLParameters(consumerProps);
}

protected Properties getConsumerProps() {
Expand Down
5 changes: 5 additions & 0 deletions flume-ng-sinks/flume-ng-kafka-sink/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,11 @@
<artifactId>flume-ng-configuration</artifactId>
</dependency>

<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka</artifactId>
</dependency>

<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka-test</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ Licensed to the Apache Software Foundation (ASF) under one or more
import org.apache.flume.conf.LogPrivacyUtil;
import org.apache.flume.formatter.output.BucketPath;
import org.apache.flume.instrumentation.kafka.KafkaSinkCounter;
import org.apache.flume.shared.kafka.KafkaSSLUtil;
import org.apache.flume.sink.AbstractSink;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.kafka.clients.producer.Callback;
Expand Down Expand Up @@ -420,6 +421,8 @@ private void setProducerProps(Context context, String bootStrapServers) {
kafkaProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, DEFAULT_VALUE_SERIAIZER);
kafkaProps.putAll(context.getSubProperties(KAFKA_PRODUCER_PREFIX));
kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootStrapServers);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
}

protected Properties getKafkaProps() {
Expand Down
4 changes: 4 additions & 0 deletions flume-ng-sources/flume-kafka-source/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,10 @@
<groupId>org.apache.flume</groupId>
<artifactId>flume-ng-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka-test</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import org.apache.flume.conf.LogPrivacyUtil;
import org.apache.flume.event.EventBuilder;
import org.apache.flume.instrumentation.kafka.KafkaSourceCounter;
import org.apache.flume.shared.kafka.KafkaSSLUtil;
import org.apache.flume.source.AbstractPollableSource;
import org.apache.flume.source.avro.AvroFlumeEvent;
import org.apache.kafka.clients.CommonClientConfigs;
Expand Down Expand Up @@ -461,6 +462,8 @@ private void setConsumerProps(Context ctx) {
}
kafkaProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
KafkaSourceConstants.DEFAULT_AUTO_COMMIT);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);
}

/**
Expand Down
51 changes: 51 additions & 0 deletions flume-shared/flume-shared-kafka/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<artifactId>flume-shared</artifactId>
<groupId>org.apache.flume</groupId>
<version>1.9.0-SNAPSHOT</version>
</parent>

<groupId>org.apache.flume.flume-shared</groupId>
<artifactId>flume-shared-kafka</artifactId>
<name>Flume Shared Kafka</name>

<dependencies>
<dependency>
<groupId>org.apache.flume</groupId>
<artifactId>flume-ng-sdk</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>${kafka.version}</version>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.flume.shared.kafka;

import org.apache.flume.util.SSLUtil;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.config.SslConfigs;
import org.apache.kafka.common.protocol.SecurityProtocol;

import java.util.Properties;

public class KafkaSSLUtil {

private KafkaSSLUtil() {
}

/**
* Adds the global keystore/truststore SSL parameters to Kafka properties
* if SSL is enabled but the keystore/truststore SSL parameters
* are not defined explicitly in Kafka properties.
*
* @param kafkaProps Kafka properties
*/
public static void addGlobalSSLParameters(Properties kafkaProps) {
if (isSSLEnabled(kafkaProps)) {
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, SSLUtil.getGlobalKeystorePath());
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, SSLUtil.getGlobalKeystorePassword());
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, SSLUtil.getGlobalKeystoreType(null));
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, SSLUtil.getGlobalTruststorePath());
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, SSLUtil.getGlobalTruststorePassword());
addGlobalSSLParameter(kafkaProps,
SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, SSLUtil.getGlobalTruststoreType(null));
}
}

private static void addGlobalSSLParameter(Properties kafkaProps,
String propName, String globalValue) {
if (!kafkaProps.containsKey(propName) && globalValue != null) {
kafkaProps.put(propName, globalValue);
}
}

private static boolean isSSLEnabled(Properties kafkaProps) {
String securityProtocol =
kafkaProps.getProperty(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG);

return securityProtocol != null &&
(securityProtocol.equals(SecurityProtocol.SSL.name) ||
securityProtocol.equals(SecurityProtocol.SASL_SSL.name));
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
/*
* 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.flume.shared.kafka;

import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.config.SslConfigs;
import org.apache.kafka.common.protocol.SecurityProtocol;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;

import java.util.Properties;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;

public class KafkaSSLUtilTest {

@Before
public void initSystemProperties() {
System.setProperty("javax.net.ssl.keyStore", "global-keystore-path");
System.setProperty("javax.net.ssl.keyStorePassword", "global-keystore-password");
System.setProperty("javax.net.ssl.keyStoreType", "global-keystore-type");
System.setProperty("javax.net.ssl.trustStore", "global-truststore-path");
System.setProperty("javax.net.ssl.trustStorePassword", "global-truststore-password");
System.setProperty("javax.net.ssl.trustStoreType", "global-truststore-type");
}

@After
public void clearSystemProperties() {
System.clearProperty("javax.net.ssl.keyStore");
System.clearProperty("javax.net.ssl.keyStorePassword");
System.clearProperty("javax.net.ssl.keyStoreType");
System.clearProperty("javax.net.ssl.trustStore");
System.clearProperty("javax.net.ssl.trustStorePassword");
System.clearProperty("javax.net.ssl.trustStoreType");
}

@Test
public void testSecurityProtocol_PLAINTEXT() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.PLAINTEXT.name);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertNoSSLParameters(kafkaProps);
}

@Test
public void testSecurityProtocol_SASL_PLAINTEXT() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_PLAINTEXT.name);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertNoSSLParameters(kafkaProps);
}

@Test
public void testSecurityProtocol_SSL() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertGlobalSSLParameters(kafkaProps);
}

@Test
public void testSecurityProtocol_SASL_SSL() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.name);

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertGlobalSSLParameters(kafkaProps);
}

@Test
public void testComponentParametersNotOverridden() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);

kafkaProps.put(
SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "component-keystore-path");
kafkaProps.put(
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "component-keystore-password");
kafkaProps.put(
SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, "component-keystore-type");
kafkaProps.put(
SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "component-truststore-path");
kafkaProps.put(
SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "component-truststore-password");
kafkaProps.put(
SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "component-truststore-type");

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertComponentSSLParameters(kafkaProps);
}

@Test
public void testEmptyGlobalParametersNotAdded() {
Properties kafkaProps = new Properties();
kafkaProps.put(
CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name);

clearSystemProperties();

KafkaSSLUtil.addGlobalSSLParameters(kafkaProps);

assertNoSSLParameters(kafkaProps);
}

private void assertNoSSLParameters(Properties kafkaProps) {
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
assertFalse(kafkaProps.containsKey(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
}

private void assertGlobalSSLParameters(Properties kafkaProps) {
assertEquals("global-keystore-path",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
assertEquals("global-keystore-password",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
assertEquals("global-keystore-type",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
assertEquals("global-truststore-path",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
assertEquals("global-truststore-password",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
assertEquals("global-truststore-type",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
}

private void assertComponentSSLParameters(Properties kafkaProps) {
assertEquals("component-keystore-path",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG));
assertEquals("component-keystore-password",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG));
assertEquals("component-keystore-type",
kafkaProps.getProperty(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG));
assertEquals("component-truststore-path",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
assertEquals("component-truststore-password",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG));
assertEquals("component-truststore-type",
kafkaProps.getProperty(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG));
}
}
Loading

0 comments on commit 493b53b

Please sign in to comment.