-
Notifications
You must be signed in to change notification settings - Fork 1.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
FLUME-3278 Handling -D keystore parameters in Kafka components
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
1 parent
965e132
commit 493b53b
Showing
11 changed files
with
329 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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> |
73 changes: 73 additions & 0 deletions
73
...e-shared/flume-shared-kafka/src/main/java/org/apache/flume/shared/kafka/KafkaSSLUtil.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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)); | ||
} | ||
|
||
} |
174 changes: 174 additions & 0 deletions
174
...ared/flume-shared-kafka/src/test/java/org/apache/flume/shared/kafka/KafkaSSLUtilTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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)); | ||
} | ||
} |
Oops, something went wrong.