You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2016/05/12 10:12:25 UTC
kafka git commit: MINOR: Change type of
StreamsConfig.BOOTSTRAP_SERVERS_CONFIG to List
Repository: kafka
Updated Branches:
refs/heads/trunk 84d17bdf2 -> 5aedde5b2
MINOR: Change type of StreamsConfig.BOOTSTRAP_SERVERS_CONFIG to List
This is an improved version of https://github.com/apache/kafka/pull/1374, where we include a unit test.
/cc ijuma and guozhangwang
Author: Guozhang Wang <wa...@gmail.com>
Author: Michael G. Noll <mi...@confluent.io>
Reviewers: Michael G. Noll <mi...@confluent.io>, Ismael Juma <is...@juma.me.uk>
Closes #1377 from miguno/streamsconfig-multiple-bootstrap-servers
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/5aedde5b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/5aedde5b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/5aedde5b
Branch: refs/heads/trunk
Commit: 5aedde5b29cc5a1adea4c65d5c44fb657f162eab
Parents: 84d17bd
Author: Guozhang Wang <wa...@gmail.com>
Authored: Thu May 12 11:12:17 2016 +0100
Committer: Ismael Juma <is...@juma.me.uk>
Committed: Thu May 12 11:12:17 2016 +0100
----------------------------------------------------------------------
.../org/apache/kafka/streams/StreamsConfig.java | 2 +-
.../apache/kafka/streams/StreamsConfigTest.java | 20 ++++++++++++++++++++
2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/5aedde5b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index fac2914..efccd7a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -120,7 +120,7 @@ public class StreamsConfig extends AbstractConfig {
Importance.HIGH,
StreamsConfig.APPLICATION_ID_DOC)
.define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value
- Type.STRING,
+ Type.LIST,
Importance.HIGH,
CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
.define(CLIENT_ID_CONFIG,
http://git-wip-us.apache.org/repos/asf/kafka/blob/5aedde5b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
index 81b406f..d7def70 100644
--- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
@@ -20,14 +20,20 @@ package org.apache.kafka.streams;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Utils;
import org.junit.Before;
import org.junit.Test;
+import java.util.Arrays;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
public class StreamsConfigTest {
@@ -84,4 +90,18 @@ public class StreamsConfigTest {
assertEquals("Should get the original string after serialization and deserialization with the configured encoding",
str, streamsConfig.valueSerde().deserializer().deserialize(topic, serializer.serialize(topic, str)));
}
+
+ @Test
+ public void shouldSupportMultipleBootstrapServers() {
+ List<String> expectedBootstrapServers = Arrays.asList("broker1:9092", "broker2:9092");
+ String bootstrapServersString = Utils.mkString(expectedBootstrapServers, ",").toString();
+ Properties props = new Properties();
+ props.put(StreamsConfig.APPLICATION_ID_CONFIG, "irrelevant");
+ props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersString);
+ StreamsConfig config = new StreamsConfig(props);
+
+ List<String> actualBootstrapServers = config.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
+ assertThat(actualBootstrapServers, equalTo(expectedBootstrapServers));
+ }
+
}