You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ju...@apache.org on 2015/10/19 22:49:15 UTC
kafka git commit: KAFKA-2656;
Remove hardcoded default key and truststores
Repository: kafka
Updated Branches:
refs/heads/trunk 964114a8a -> 674543525
KAFKA-2656; Remove hardcoded default key and truststores
Removed default hardcoded keystore and truststore in /tmp so that default JVM keystore/truststore may be used when keystore/truststore is not specified in Kafka server or client properties
Author: Rajini Sivaram <ra...@googlemail.com>
Reviewers: Ismael Juma <is...@juma.me.uk>, Jun Rao <ju...@gmail.com>
Closes #312 from rajinisivaram/KAFKA-2656
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/67454352
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/67454352
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/67454352
Branch: refs/heads/trunk
Commit: 6745435251feab3085056cd2f8c9c400da849048
Parents: 964114a
Author: Rajini Sivaram <ra...@googlemail.com>
Authored: Mon Oct 19 13:49:09 2015 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Oct 19 13:49:09 2015 -0700
----------------------------------------------------------------------
.../kafka/clients/consumer/ConsumerConfig.java | 4 ++--
.../kafka/clients/producer/ProducerConfig.java | 4 ++--
.../apache/kafka/common/config/SSLConfigs.java | 6 -----
.../kafka/common/security/ssl/SSLFactory.java | 25 +++++++-------------
.../main/scala/kafka/server/KafkaConfig.scala | 15 ++++--------
5 files changed, 17 insertions(+), 37 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/67454352/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index bcc143c..1042208 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -297,8 +297,8 @@ public class ConsumerConfig extends AbstractConfig {
.define(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEY_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_TRUSTSTORE_TYPE_DOC)
- .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC)
- .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC)
+ .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC, false)
+ .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67454352/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index c3dbd10..4badd33 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -274,8 +274,8 @@ public class ProducerConfig extends AbstractConfig {
.define(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEYSTORE_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_KEY_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, Importance.MEDIUM, SSLConfigs.SSL_TRUSTSTORE_TYPE_DOC)
- .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC)
- .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC)
+ .define(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_LOCATION_DOC, false)
+ .define(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, SSLConfigs.SSL_TRUSTSTORE_PASSWORD_DOC, false)
.define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67454352/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java
index 77bb583..0fed961 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/SSLConfigs.java
@@ -66,11 +66,9 @@ public class SSLConfigs {
public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location";
public static final String SSL_TRUSTSTORE_LOCATION_DOC = "The location of the trust store file. ";
- public static final String DEFAULT_TRUSTSTORE_LOCATION = "/tmp/ssl.truststore.jks";
public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "ssl.truststore.password";
public static final String SSL_TRUSTSTORE_PASSWORD_DOC = "The password for the trust store file. ";
- public static final String DEFAULT_TRUSTSTORE_PASSWORD = "truststore_password";
public static final String SSL_KEYMANAGER_ALGORITHM_CONFIG = "ssl.keymanager.algorithm";
public static final String SSL_KEYMANAGER_ALGORITHM_DOC = "The algorithm used by key manager factory for SSL connections. "
@@ -95,8 +93,4 @@ public class SSLConfigs {
+ " unlike requested , if this option is set client can choose not to provide authentication information about itself"
+ " <li><code>ssl.client.auth=none</code> This means client authentication is not needed.";
- public static final String SSL_NEED_CLIENT_AUTH_DOC = "It can be REQUESTED . "
- + "Default value is false";
- public static final Boolean DEFAULT_SSL_NEED_CLIENT_AUTH = false;
-
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67454352/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
index b3e0895..b291409 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
@@ -83,12 +83,10 @@ public class SSLFactory implements Configurable {
this.kmfAlgorithm = (String) configs.get(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG);
this.tmfAlgorithm = (String) configs.get(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG);
- if (checkKeyStoreConfigs(configs)) {
- createKeystore((String) configs.get(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG),
- (String) configs.get(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG),
- (String) configs.get(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG),
- (String) configs.get(SSLConfigs.SSL_KEY_PASSWORD_CONFIG));
- }
+ createKeystore((String) configs.get(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG),
+ (String) configs.get(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG),
+ (String) configs.get(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG),
+ (String) configs.get(SSLConfigs.SSL_KEY_PASSWORD_CONFIG));
createTruststore((String) configs.get(SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG),
(String) configs.get(SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG),
@@ -157,9 +155,9 @@ public class SSLFactory implements Configurable {
private void createKeystore(String type, String path, String password, String keyPassword) {
if (path == null && password != null) {
- throw new KafkaException("SSL key store password is not specified.");
- } else if (path != null && password == null) {
throw new KafkaException("SSL key store is not specified, but key store password is specified.");
+ } else if (path != null && password == null) {
+ throw new KafkaException("SSL key store is specified, but key store password is not specified.");
} else if (path != null && password != null) {
this.keystore = new SecurityStore(type, path, password);
this.keyPassword = keyPassword;
@@ -168,21 +166,14 @@ public class SSLFactory implements Configurable {
private void createTruststore(String type, String path, String password) {
if (path == null && password != null) {
- throw new KafkaException("SSL key store password is not specified.");
+ throw new KafkaException("SSL trust store is not specified, but trust store password is specified.");
} else if (path != null && password == null) {
- throw new KafkaException("SSL key store is not specified, but key store password is specified.");
+ throw new KafkaException("SSL trust store is specified, but trust store password is not specified.");
} else if (path != null && password != null) {
this.truststore = new SecurityStore(type, path, password);
}
}
- private boolean checkKeyStoreConfigs(Map<String, ?> configs) {
- return configs.containsKey(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG) &&
- configs.containsKey(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG) &&
- configs.containsKey(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) &&
- configs.containsKey(SSLConfigs.SSL_KEY_PASSWORD_CONFIG);
- }
-
private class SecurityStore {
private final String type;
private final String path;
http://git-wip-us.apache.org/repos/asf/kafka/blob/67454352/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index d9d87cd..913d49b 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -160,12 +160,7 @@ object Defaults {
val SSLProtocol = SSLConfigs.DEFAULT_SSL_PROTOCOL
val SSLEnabledProtocols = SSLConfigs.DEFAULT_ENABLED_PROTOCOLS
val SSLKeystoreType = SSLConfigs.DEFAULT_SSL_KEYSTORE_TYPE
- val SSLKeystoreLocation = "/tmp/ssl.keystore.jks"
- val SSLKeystorePassword = "keystore_password"
- val SSLKeyPassword = "key_password"
val SSLTruststoreType = SSLConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE
- val SSLTruststoreLocation = SSLConfigs.DEFAULT_TRUSTSTORE_LOCATION
- val SSLTruststorePassword = SSLConfigs.DEFAULT_TRUSTSTORE_PASSWORD
val SSLKeyManagerAlgorithm = SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM
val SSLTrustManagerAlgorithm = SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM
val SSLClientAuthRequired = "required"
@@ -637,12 +632,12 @@ object KafkaConfig {
.define(SSLProviderProp, STRING, MEDIUM, SSLProviderDoc, false)
.define(SSLEnabledProtocolsProp, LIST, Defaults.SSLEnabledProtocols, MEDIUM, SSLEnabledProtocolsDoc)
.define(SSLKeystoreTypeProp, STRING, Defaults.SSLKeystoreType, MEDIUM, SSLKeystoreTypeDoc)
- .define(SSLKeystoreLocationProp, STRING, Defaults.SSLKeystoreLocation, MEDIUM, SSLKeystoreLocationDoc)
- .define(SSLKeystorePasswordProp, STRING, Defaults.SSLKeystorePassword, MEDIUM, SSLKeystorePasswordDoc)
- .define(SSLKeyPasswordProp, STRING, Defaults.SSLKeyPassword, MEDIUM, SSLKeyPasswordDoc)
+ .define(SSLKeystoreLocationProp, STRING, MEDIUM, SSLKeystoreLocationDoc, false)
+ .define(SSLKeystorePasswordProp, STRING, MEDIUM, SSLKeystorePasswordDoc, false)
+ .define(SSLKeyPasswordProp, STRING, MEDIUM, SSLKeyPasswordDoc, false)
.define(SSLTruststoreTypeProp, STRING, Defaults.SSLTruststoreType, MEDIUM, SSLTruststoreTypeDoc)
- .define(SSLTruststoreLocationProp, STRING, Defaults.SSLTruststoreLocation, MEDIUM, SSLTruststoreLocationDoc)
- .define(SSLTruststorePasswordProp, STRING, Defaults.SSLTruststorePassword, MEDIUM, SSLTruststorePasswordDoc)
+ .define(SSLTruststoreLocationProp, STRING, MEDIUM, SSLTruststoreLocationDoc, false)
+ .define(SSLTruststorePasswordProp, STRING, MEDIUM, SSLTruststorePasswordDoc, false)
.define(SSLKeyManagerAlgorithmProp, STRING, Defaults.SSLKeyManagerAlgorithm, MEDIUM, SSLKeyManagerAlgorithmDoc)
.define(SSLTrustManagerAlgorithmProp, STRING, Defaults.SSLTrustManagerAlgorithm, MEDIUM, SSLTrustManagerAlgorithmDoc)
.define(SSLClientAuthProp, STRING, Defaults.SSLClientAuth, in(Defaults.SSLClientAuthRequired, Defaults.SSLClientAuthRequested, Defaults.SSLClientAuthNone), MEDIUM, SSLClientAuthDoc)