From aaa7a0f44ad0bb77fbf9dfaa18b0d6fc48f6a612 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 16 Mar 2015 08:54:14 -0700 Subject: [PATCH 1/3] squashing multi-broker-endpoint patches --- .../apache/kafka/common/protocol/ApiVersion.java | 65 +++++++++++ .../kafka/common/protocol/SecurityProtocol.java | 59 ++++++++++ .../main/scala/kafka/cluster/BrokerEndPoint.scala | 67 +++++++++++ core/src/main/scala/kafka/cluster/EndPoint.scala | 72 ++++++++++++ .../scala/kafka/cluster/SecurityProtocol.scala | 28 +++++ .../BrokerEndPointNotAvailableException.scala | 22 ++++ .../test/scala/unit/kafka/cluster/BrokerTest.scala | 129 +++++++++++++++++++++ system_test/run_all.sh | 10 ++ system_test/run_all_replica.sh | 10 ++ 9 files changed, 462 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java create mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java create mode 100644 core/src/main/scala/kafka/cluster/BrokerEndPoint.scala create mode 100644 core/src/main/scala/kafka/cluster/EndPoint.scala create mode 100644 core/src/main/scala/kafka/cluster/SecurityProtocol.scala create mode 100644 core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala create mode 100644 core/src/test/scala/unit/kafka/cluster/BrokerTest.scala create mode 100755 system_test/run_all.sh create mode 100755 system_test/run_all_replica.sh diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java new file mode 100644 index 0000000..fdd8585 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java @@ -0,0 +1,65 @@ +/** + * 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.kafka.common.protocol; + +/** + * This class contains the different Kafka versions. + * Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves. + * This is only for intra-broker communications - when communicating with clients, the client decides on the API version. + * + * Note that ORDER MATTERS in the enum. + * We consider version A as newer than B if it appears later in the list of constants here. + * If you add new versions, add them in the correct chronological release order. + */ +public enum ApiVersion { + KAFKA_082("0.8.2.X"), + KAFKA_083("0.8.3.X"); + + private final String version; + + public boolean onOrAfter(ApiVersion other) { + return compareTo(other) >= 0; + } + + private ApiVersion(final String version) { + this.version = version; + } + + /* Parse user readable version number. This assumes the convention of 0.8.2.0, 0.8.2.1, 0.8.3.0, 0.9.0.0, etc. + * We are assuming that we will never change API in bug-fix versions (i.e. 0.8.2.1 will have same API as 0.8.2.0) + * */ + public static ApiVersion parseConfig(String version) { + String[] vals = version.split("\\."); + StringBuilder parsed = new StringBuilder(); + parsed.append("KAFKA_"); + + // We only care about the first 3 version digits. Others don't impact the API + parsed.append(vals[0]).append(vals[1]).append(vals[2]); + return ApiVersion.valueOf(parsed.toString()); + } + + public static ApiVersion getLatestVersion() { + ApiVersion[] values = ApiVersion.values(); + return values[values.length - 1]; + } + + @Override + public String toString() { + return version; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java new file mode 100644 index 0000000..276939a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -0,0 +1,59 @@ +/** + * 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.kafka.common.protocol; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public enum SecurityProtocol { + /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */ + TRACE(0, "TRACE"), + /** Un-authenticated, non-encrypted channel */ + PLAINTEXT(1, "PLAINTEXT"); + + private static Map codeToSecurityProtocol = new HashMap(); + private static List names = new ArrayList(); + + static { + for (SecurityProtocol proto: SecurityProtocol.values()) { + codeToSecurityProtocol.put(proto.id, proto); + names.add(proto.name); + } + } + + /** The permanent and immutable id of a security protocol -- this can't change, and must match kafka.cluster.SecurityProtocol */ + public final short id; + + /** A name of the security protocol. This may be used by client configuration. */ + public final String name; + + private SecurityProtocol(int id, String name) { + this.id = (short) id; + this.name = name; + } + + public static String getName(int id) { + return codeToSecurityProtocol.get(id).name; + } + + public static List getNames() { + return names; + } + +} diff --git a/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala new file mode 100644 index 0000000..22dba18 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/BrokerEndPoint.scala @@ -0,0 +1,67 @@ +/** + * 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 kafka.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import org.apache.kafka.common.utils.Utils._ + +object BrokerEndpoint { + def createBrokerEndPoint(brokerId: Int, connectionString: String): BrokerEndpoint = { + + // BrokerEndPoint URI is host:port or [ipv6_host]:port + // Note that unlike EndPoint (or listener) this URI has no security information. + val uriParseExp = """\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r + + connectionString match { + case uriParseExp(host, port) => new BrokerEndpoint(brokerId, host, port.toInt) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } + } + + def readFrom(buffer: ByteBuffer): BrokerEndpoint = { + val brokerId = buffer.getInt() + val host = readShortString(buffer) + val port = buffer.getInt() + BrokerEndpoint(brokerId, host, port) + } +} + +/** + * BrokerEndpoint is used to connect to specific host:port pair. + * It is typically used by clients (or brokers when connecting to other brokers) + * and contains no information about the security protocol used on the connection. + * Clients should know which security protocol to use from configuration. + * This allows us to keep the wire protocol with the clients unchanged where the protocol is not needed. + */ +case class BrokerEndpoint(id: Int, host: String, port: Int) { + + def connectionString(): String = formatAddress(host, port) + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(id) + writeShortString(buffer, host) + buffer.putInt(port) + } + + def sizeInBytes: Int = + 4 + /* broker Id */ + 4 + /* port */ + shortStringLength(host) +} diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala new file mode 100644 index 0000000..1d853b1 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -0,0 +1,72 @@ +/* + * 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 kafka.cluster + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.KafkaException +import kafka.cluster.SecurityProtocol._ + +object EndPoint { + + def readFrom(buffer: ByteBuffer): EndPoint = { + val port = buffer.getInt() + val host = readShortString(buffer) + val protocol = buffer.getShort() + EndPoint(host, port, SecurityProtocol(protocol)) + } + + /** + * Create EndPoint object from connectionString + * @param connectionString the format is protocol://host:port or protocol://[ipv6 host]:port + * for example: PLAINTEXT://myhost:9092 or PLAINTEXT://[::1]:9092 + * Host can be empty (PLAINTEXT://:9092) in which case we'll bind to default interface + * @return + */ + def createEndPoint(connectionString: String): EndPoint = { + val uriParseExp = """^(.*)://\[?([0-9a-z\-.:]*)\]?:([0-9]+)""".r + connectionString match { + case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.withName(protocol)) + case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.withName(protocol)) + case _ => throw new KafkaException("Unable to parse " + connectionString + " to a broker endpoint") + } + } +} + +/** + * Part of the broker definition - matching host/port pair to a protocol + */ +case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { + + override def toString: String = { + val hostStr = if (host == null || host.contains(":")) "[" + host + "]" else host + protocolType + "://" + hostStr + ":" + port + } + + def writeTo(buffer: ByteBuffer): Unit = { + buffer.putInt(port) + writeShortString(buffer, host) + buffer.putShort(protocolType.id.toShort) + } + + def sizeInBytes: Int = + 4 + /* port */ + shortStringLength(host) + + 2 /* protocol id */ +} diff --git a/core/src/main/scala/kafka/cluster/SecurityProtocol.scala b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala new file mode 100644 index 0000000..a732385 --- /dev/null +++ b/core/src/main/scala/kafka/cluster/SecurityProtocol.scala @@ -0,0 +1,28 @@ +/* + * 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 kafka.cluster + + +object SecurityProtocol extends Enumeration { + + type SecurityProtocol = Value + /* TRACE is used for testing non-default protocol */ + val TRACE = Value + val PLAINTEXT = Value +} + diff --git a/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala new file mode 100644 index 0000000..455d8c6 --- /dev/null +++ b/core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala @@ -0,0 +1,22 @@ +/* + * 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 kafka.common + +class BrokerEndPointNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala new file mode 100644 index 0000000..3df8aca --- /dev/null +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -0,0 +1,129 @@ +/* + * 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 kafka.cluster + +import java.nio.ByteBuffer + +import kafka.utils.Logging +import org.junit.Test +import org.scalatest.junit.JUnit3Suite + +import scala.collection.mutable + +class BrokerTest extends JUnit3Suite with Logging { + + @Test + def testSerDe() = { + + val endpoint = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val listEndPoints = Map(SecurityProtocol.PLAINTEXT -> endpoint) + val origBroker = new Broker(1, listEndPoints) + val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) + + origBroker.writeTo(brokerBytes) + + val newBroker = Broker.readFrom(brokerBytes.flip().asInstanceOf[ByteBuffer]) + assert(origBroker == newBroker) + } + + @Test + def testHashAndEquals() = { + val endpoint1 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint2 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) + val endpoint3 = new EndPoint("myhost", 1111, SecurityProtocol.PLAINTEXT) + val endpoint4 = new EndPoint("other", 1111, SecurityProtocol.PLAINTEXT) + val broker1 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint1)) + val broker2 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint2)) + val broker3 = new Broker(2, Map(SecurityProtocol.PLAINTEXT -> endpoint3)) + val broker4 = new Broker(1, Map(SecurityProtocol.PLAINTEXT -> endpoint4)) + + assert(broker1 == broker2) + assert(broker1 != broker3) + assert(broker1 != broker4) + assert(broker1.hashCode() == broker2.hashCode()) + assert(broker1.hashCode() != broker3.hashCode()) + assert(broker1.hashCode() != broker4.hashCode()) + + val hashmap = new mutable.HashMap[Broker, Int]() + hashmap.put(broker1, 1) + assert(hashmap.getOrElse(broker1, -1) == 1) + } + + @Test + def testFromJSON() = { + val brokerInfoStr = "{\"version\":2," + + "\"host\":\"localhost\"," + + "\"port\":9092," + + "\"jmx_port\":9999," + + "\"timestamp\":\"1416974968782\"," + + "\"endpoints\":[\"PLAINTEXT://localhost:9092\"]}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "localhost") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9092) + } + + @Test + def testFromOldJSON() = { + val brokerInfoStr = "{\"jmx_port\":-1,\"timestamp\":\"1420485325400\",\"host\":\"172.16.8.243\",\"version\":1,\"port\":9091}" + val broker = Broker.createBroker(1, brokerInfoStr) + assert(broker.id == 1) + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host == "172.16.8.243") + assert(broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port == 9091) + } + + @Test + def testBrokerEndpointFromURI() = { + var connectionString = "localhost:9092" + var endpoint = BrokerEndpoint.createBrokerEndPoint(1, connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + // also test for ipv6 + connectionString = "[::1]:9092" + endpoint = BrokerEndpoint.createBrokerEndPoint(1, connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) + } + + @Test + def testEndpointFromURI() = { + var connectionString = "PLAINTEXT://localhost:9092" + var endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "localhost") + assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://localhost:9092") + // also test for default bind + connectionString = "PLAINTEXT://:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == null) + assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://[null]:9092") + // also test for ipv6 + connectionString = "PLAINTEXT://[::1]:9092" + endpoint = EndPoint.createEndPoint(connectionString) + assert(endpoint.host == "::1") + assert(endpoint.port == 9092) + assert(endpoint.toString == "PLAINTEXT://[::1]:9092") + } + + + + + + +} diff --git a/system_test/run_all.sh b/system_test/run_all.sh new file mode 100755 index 0000000..60cbc59 --- /dev/null +++ b/system_test/run_all.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +my_ts=`date +"%s"` + +cp testcase_to_run.json testcase_to_run.json_${my_ts} +cp testcase_to_run_all.json testcase_to_run.json + +python -B system_test_runner.py + + diff --git a/system_test/run_all_replica.sh b/system_test/run_all_replica.sh new file mode 100755 index 0000000..22aaa60 --- /dev/null +++ b/system_test/run_all_replica.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +my_ts=`date +"%s"` + +cp testcase_to_run.json testcase_to_run.json_${my_ts} +cp testcase_to_run_all_replica.json testcase_to_run.json + +python -B system_test_runner.py + + -- 1.9.5 (Apple Git-50.3) From deed3e55ddb2c9ee688472aba4eef59b658bf08d Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 16 Mar 2015 09:40:44 -0700 Subject: [PATCH 2/3] forgot rest of patch --- .../kafka/clients/producer/ProducerConfig.java | 12 +- .../java/org/apache/kafka/common/utils/Utils.java | 4 +- .../org/apache/kafka/common/utils/UtilsTest.java | 4 +- config/server.properties | 4 +- core/src/main/scala/kafka/admin/AdminUtils.scala | 21 ++-- core/src/main/scala/kafka/admin/TopicCommand.scala | 4 +- .../scala/kafka/api/ConsumerMetadataResponse.scala | 8 +- .../main/scala/kafka/api/LeaderAndIsrRequest.scala | 10 +- core/src/main/scala/kafka/api/TopicMetadata.scala | 20 ++-- .../scala/kafka/api/TopicMetadataResponse.scala | 6 +- .../scala/kafka/api/UpdateMetadataRequest.scala | 32 ++++-- core/src/main/scala/kafka/client/ClientUtils.scala | 30 ++--- core/src/main/scala/kafka/cluster/Broker.scala | 112 +++++++++++++++---- .../main/scala/kafka/consumer/ConsumerConfig.scala | 7 ++ .../kafka/consumer/ConsumerFetcherManager.scala | 9 +- .../kafka/consumer/ConsumerFetcherThread.scala | 4 +- .../consumer/ZookeeperConsumerConnector.scala | 3 +- .../controller/ControllerChannelManager.scala | 12 +- .../scala/kafka/controller/KafkaController.scala | 6 +- .../kafka/javaapi/ConsumerMetadataResponse.scala | 5 +- .../main/scala/kafka/javaapi/TopicMetadata.scala | 8 +- .../main/scala/kafka/network/RequestChannel.scala | 6 +- .../main/scala/kafka/network/SocketServer.scala | 58 +++++++--- .../main/scala/kafka/producer/ProducerPool.scala | 11 +- .../kafka/server/AbstractFetcherManager.scala | 8 +- .../scala/kafka/server/AbstractFetcherThread.scala | 5 +- core/src/main/scala/kafka/server/KafkaApis.scala | 12 +- core/src/main/scala/kafka/server/KafkaConfig.scala | 106 ++++++++++++++++-- .../main/scala/kafka/server/KafkaHealthcheck.scala | 24 ++-- core/src/main/scala/kafka/server/KafkaServer.scala | 105 ++++++++--------- .../main/scala/kafka/server/MetadataCache.scala | 25 +++-- .../scala/kafka/server/ReplicaFetcherManager.scala | 4 +- .../scala/kafka/server/ReplicaFetcherThread.scala | 4 +- .../main/scala/kafka/server/ReplicaManager.scala | 4 +- .../scala/kafka/tools/ConsumerOffsetChecker.scala | 3 +- .../kafka/tools/ReplicaVerificationTool.scala | 6 +- .../scala/kafka/tools/SimpleConsumerShell.scala | 18 ++- .../main/scala/kafka/tools/UpdateOffsetsInZK.scala | 5 +- core/src/main/scala/kafka/utils/Utils.scala | 8 ++ core/src/main/scala/kafka/utils/ZkUtils.scala | 36 ++++-- .../kafka/api/ProducerFailureHandlingTest.scala | 10 +- .../integration/kafka/api/ProducerSendTest.scala | 8 +- .../test/scala/other/kafka/TestOffsetManager.scala | 5 +- .../test/scala/unit/kafka/KafkaConfigTest.scala | 8 +- .../scala/unit/kafka/admin/AddPartitionsTest.scala | 10 +- .../api/RequestResponseSerializationTest.scala | 69 ++++++++++-- .../unit/kafka/consumer/ConsumerIteratorTest.scala | 2 +- .../scala/unit/kafka/integration/FetcherTest.scala | 5 +- .../kafka/integration/KafkaServerTestHarness.scala | 9 +- .../unit/kafka/integration/TopicMetadataTest.scala | 14 +-- core/src/test/scala/unit/kafka/log/LogTest.scala | 4 +- .../unit/kafka/network/SocketServerTest.scala | 57 ++++++++-- .../unit/kafka/producer/AsyncProducerTest.scala | 8 +- .../unit/kafka/producer/SyncProducerTest.scala | 25 +++-- .../unit/kafka/server/AdvertiseBrokerTest.scala | 11 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 7 ++ .../scala/unit/kafka/server/KafkaConfigTest.scala | 124 +++++++++++++++++---- .../unit/kafka/server/LeaderElectionTest.scala | 7 +- .../scala/unit/kafka/server/LogOffsetTest.scala | 2 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 16 +-- .../testcase_0001/testcase_0001_properties.json | 8 +- .../testcase_0002/testcase_0002_properties.json | 8 +- .../testcase_0003/testcase_0003_properties.json | 8 +- .../testcase_0004/testcase_0004_properties.json | 8 +- .../testcase_0005/testcase_0005_properties.json | 8 +- .../testcase_0006/testcase_0006_properties.json | 8 +- .../testcase_0007/testcase_0007_properties.json | 8 +- .../testcase_0008/testcase_0008_properties.json | 8 +- .../testcase_0009/testcase_0009_properties.json | 8 +- .../testcase_0101/testcase_0101_properties.json | 8 +- .../testcase_0102/testcase_0102_properties.json | 8 +- .../testcase_0103/testcase_0103_properties.json | 8 +- .../testcase_0104/testcase_0104_properties.json | 8 +- .../testcase_0105/testcase_0105_properties.json | 8 +- .../testcase_0106/testcase_0106_properties.json | 8 +- .../testcase_0107/testcase_0107_properties.json | 8 +- .../testcase_0108/testcase_0108_properties.json | 8 +- .../testcase_0109/testcase_0109_properties.json | 8 +- .../testcase_1/testcase_1_properties.json | 6 +- system_test/utils/kafka_system_test_utils.py | 1 + 80 files changed, 889 insertions(+), 432 deletions(-) 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 fa9daae..0607ef3 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 @@ -169,6 +169,10 @@ public class ProducerConfig extends AbstractConfig { public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + /** security.protocol */ + public static final String SECURITY_PROTOCOL = "security.protocol"; + private static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT is supported. SSL and Kerberos are planned for the near future"; + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) @@ -218,7 +222,13 @@ public class ProducerConfig extends AbstractConfig { Importance.LOW, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) + .define(SECURITY_PROTOCOL, + Type.STRING, + "PLAINTEXT", + in("PLAINTEXT"), + Importance.MEDIUM, + SECURITY_PROTOCOL_DOC); } public static Map addSerializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 920b51a..39e8d7c 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -27,7 +27,9 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); + // This matches URIs of formats: host:port and protocol:\\host:port + // IPv6 is supported with [ip] pattern + private static final Pattern HOST_PORT_PATTERN = Pattern.compile(".*?\\[?([0-9a-z\\-.:]*)\\]?:([0-9]+)"); public static final String NL = System.getProperty("line.separator"); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index c899813..4b706d7 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -31,9 +31,9 @@ public class UtilsTest { @Test public void testGetHost() { assertEquals("127.0.0.1", getHost("127.0.0.1:8000")); - assertEquals("mydomain.com", getHost("mydomain.com:8080")); + assertEquals("mydomain.com", getHost("PLAINTEXT://mydomain.com:8080")); assertEquals("::1", getHost("[::1]:1234")); - assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); + assertEquals("2001:db8:85a3:8d3:1319:8a2e:370:7348", getHost("PLAINTEXT://[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678")); } @Test diff --git a/config/server.properties b/config/server.properties index 1614260..80ee2fc 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,8 +21,10 @@ broker.id=0 ############################# Socket Server Settings ############################# +listeners=PLAINTEXT://:9092 + # The port the socket server listens on -port=9092 +#port=9092 # Hostname the broker will bind to. If not set, the server will bind to all interfaces #host.name=localhost diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index b700110..3799e86 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -18,14 +18,15 @@ package kafka.admin import kafka.common._ -import kafka.cluster.Broker +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{BrokerEndpoint, Broker, SecurityProtocol} + import kafka.log.LogConfig import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} import java.util.Random import java.util.Properties -import scala.Some import scala.Predef._ import scala.collection._ import mutable.ListBuffer @@ -341,7 +342,9 @@ object AdminUtils extends Logging { topics.map(topic => fetchTopicMetadataFromZk(topic, zkClient, cachedBrokerInfo)) } - private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker]): TopicMetadata = { + + + private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): TopicMetadata = { if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) { val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) @@ -352,22 +355,22 @@ object AdminUtils extends Logging { val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) - var leaderInfo: Option[Broker] = None - var replicaInfo: Seq[Broker] = Nil - var isrInfo: Seq[Broker] = Nil + var leaderInfo: Option[BrokerEndpoint] = None + var replicaInfo: Seq[BrokerEndpoint] = Nil + var isrInfo: Seq[BrokerEndpoint] = Nil try { leaderInfo = leader match { case Some(l) => try { - Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head) + Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol)) } catch { case e: Throwable => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e) } case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } try { - replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas.map(id => id.toInt)) - isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas) + replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas).map(_.getBrokerEndPoint(protocol)) + isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol)) } catch { case e: Throwable => throw new ReplicaNotAvailableException(e) } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index f400b71..20126fb 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -196,9 +196,7 @@ object TopicCommand { } } } - - def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(broker.host, broker.port) + ")" - + def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 24aaf95..a3587e4 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -18,18 +18,18 @@ package kafka.api import java.nio.ByteBuffer -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.common.ErrorMapping object ConsumerMetadataResponse { val CurrentVersion = 0 - private val NoBrokerOpt = Some(Broker(id = -1, host = "", port = -1)) + private val NoBrokerOpt = Some(BrokerEndpoint(id = -1, host = "", port = -1)) def readFrom(buffer: ByteBuffer) = { val correlationId = buffer.getInt val errorCode = buffer.getShort - val broker = Broker.readFrom(buffer) + val broker = BrokerEndpoint.readFrom(buffer) val coordinatorOpt = if (errorCode == ErrorMapping.NoError) Some(broker) else @@ -40,7 +40,7 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int) +case class ConsumerMetadataResponse (coordinatorOpt: Option[BrokerEndpoint], errorCode: Short, correlationId: Int) extends RequestOrResponse() { def sizeInBytes = diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 4ff7e8f..bf93632 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -21,7 +21,7 @@ package kafka.api import java.nio._ import kafka.utils._ import kafka.api.ApiUtils._ -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.controller.LeaderIsrAndControllerEpoch import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.common.ErrorMapping @@ -120,9 +120,9 @@ object LeaderAndIsrRequest { } val leadersCount = buffer.getInt - var leaders = Set[Broker]() + var leaders = Set[BrokerEndpoint]() for (i <- 0 until leadersCount) - leaders += Broker.readFrom(buffer) + leaders += BrokerEndpoint.readFrom(buffer) new LeaderAndIsrRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, leaders) } @@ -134,10 +134,10 @@ case class LeaderAndIsrRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], - leaders: Set[Broker]) + leaders: Set[BrokerEndpoint]) extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) { - def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int, + def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[BrokerEndpoint], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { this(LeaderAndIsrRequest.CurrentVersion, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos, leaders) diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index 0190076..6de447d 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -17,18 +17,17 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.common._ -import org.apache.kafka.common.utils.Utils._ object TopicMetadata { val NoLeaderNodeId = -1 - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): TopicMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndpoint]): TopicMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val topic = readShortString(buffer) val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue)) @@ -89,7 +88,7 @@ case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadat object PartitionMetadata { - def readFrom(buffer: ByteBuffer, brokers: Map[Int, Broker]): PartitionMetadata = { + def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndpoint]): PartitionMetadata = { val errorCode = readShortInRange(buffer, "error code", (-1, Short.MaxValue)) val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */ val leaderId = buffer.getInt @@ -110,9 +109,9 @@ object PartitionMetadata { } case class PartitionMetadata(partitionId: Int, - val leader: Option[Broker], - replicas: Seq[Broker], - isr: Seq[Broker] = Seq.empty, + val leader: Option[BrokerEndpoint], + replicas: Seq[BrokerEndpoint], + isr: Seq[BrokerEndpoint] = Seq.empty, errorCode: Short = ErrorMapping.NoError) extends Logging { def sizeInBytes: Int = { 2 /* error code */ + @@ -142,14 +141,13 @@ case class PartitionMetadata(partitionId: Int, override def toString(): String = { val partitionMetadataString = new StringBuilder partitionMetadataString.append("\tpartition " + partitionId) - partitionMetadataString.append("\tleader: " + (if(leader.isDefined) formatBroker(leader.get) else "none")) - partitionMetadataString.append("\treplicas: " + replicas.map(formatBroker).mkString(",")) - partitionMetadataString.append("\tisr: " + isr.map(formatBroker).mkString(",")) + partitionMetadataString.append("\tleader: " + (if(leader.isDefined) leader.get.toString else "none")) + partitionMetadataString.append("\treplicas: " + replicas.mkString(",")) + partitionMetadataString.append("\tisr: " + isr.mkString(",")) partitionMetadataString.append("\tisUnderReplicated: %s".format(if(isr.size < replicas.size) "true" else "false")) partitionMetadataString.toString() } - private def formatBroker(broker: Broker) = broker.id + " (" + formatAddress(broker.host, broker.port) + ")" } diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index 92ac4e6..776b604 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -17,7 +17,7 @@ package kafka.api -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import java.nio.ByteBuffer object TopicMetadataResponse { @@ -25,7 +25,7 @@ object TopicMetadataResponse { def readFrom(buffer: ByteBuffer): TopicMetadataResponse = { val correlationId = buffer.getInt val brokerCount = buffer.getInt - val brokers = (0 until brokerCount).map(_ => Broker.readFrom(buffer)) + val brokers = (0 until brokerCount).map(_ => BrokerEndpoint.readFrom(buffer)) val brokerMap = brokers.map(b => (b.id, b)).toMap val topicCount = buffer.getInt val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) @@ -33,7 +33,7 @@ object TopicMetadataResponse { } } -case class TopicMetadataResponse(brokers: Seq[Broker], +case class TopicMetadataResponse(brokers: Seq[BrokerEndpoint], topicsMetadata: Seq[TopicMetadata], correlationId: Int) extends RequestOrResponse() { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 530982e..b389785 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -17,15 +17,15 @@ package kafka.api import java.nio.ByteBuffer +import kafka.cluster.{SecurityProtocol, BrokerEndpoint, Broker} import kafka.api.ApiUtils._ -import kafka.cluster.Broker -import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.common.{KafkaException, ErrorMapping, TopicAndPartition} import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import collection.Set object UpdateMetadataRequest { - val CurrentVersion = 0.shortValue + val CurrentVersion = 1.shortValue val IsInit: Boolean = true val NotInit: Boolean = false val DefaultAckTimeout: Int = 1000 @@ -48,7 +48,13 @@ object UpdateMetadataRequest { } val numAliveBrokers = buffer.getInt - val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + + val aliveBrokers = versionId match { + case 0 => for(i <- 0 until numAliveBrokers) yield new Broker(BrokerEndpoint.readFrom(buffer),SecurityProtocol.PLAINTEXT) + case 1 => for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, partitionStateInfos.toMap, aliveBrokers.toSet) } @@ -82,7 +88,12 @@ case class UpdateMetadataRequest (versionId: Short, value.writeTo(buffer) } buffer.putInt(aliveBrokers.size) - aliveBrokers.foreach(_.writeTo(buffer)) + + versionId match { + case 0 => aliveBrokers.foreach(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).writeTo(buffer)) + case 1 => aliveBrokers.foreach(_.writeTo(buffer)) + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } } def sizeInBytes(): Int = { @@ -96,8 +107,15 @@ case class UpdateMetadataRequest (versionId: Short, for((key, value) <- partitionStateInfos) size += (2 + key.topic.length) /* topic */ + 4 /* partition */ + value.sizeInBytes /* partition state info */ size += 4 /* number of alive brokers in the cluster */ - for(broker <- aliveBrokers) - size += broker.sizeInBytes /* broker info */ + + versionId match { + case 0 => for(broker <- aliveBrokers) + size += broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).sizeInBytes /* broker info */ + case 1 => for(broker <- aliveBrokers) + size += broker.sizeInBytes + case v => throw new KafkaException( "Version " + v.toString + " is invalid for UpdateMetadataRequest. Valid versions are 0 or 1.") + } + size } diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index ebba87f..614d4b8 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -18,17 +18,17 @@ import scala.collection._ import kafka.cluster._ +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.api._ import kafka.producer._ import kafka.common.{ErrorMapping, KafkaException} import kafka.utils.{Utils, Logging} import java.util.Properties import util.Random - import kafka.network.BlockingChannel - import kafka.utils.ZkUtils._ - import org.I0Itec.zkclient.ZkClient - import java.io.IOException -import org.apache.kafka.common.utils.Utils.{getHost, getPort} +import kafka.network.BlockingChannel +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.ZkClient +import java.io.IOException /** * Helper functions common to clients (producer, consumer, or admin) @@ -42,7 +42,7 @@ object ClientUtils extends Logging{ * @param producerConfig The producer's config * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndpoint], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { var fetchMetaDataSucceeded: Boolean = false var i: Int = 0 val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, producerConfig.clientId, topics.toSeq) @@ -83,7 +83,7 @@ object ClientUtils extends Logging{ * @param clientId The client's identifier * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int, + def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndpoint], clientId: String, timeoutMs: Int, correlationId: Int = 0): TopicMetadataResponse = { val props = new Properties() props.put("metadata.broker.list", brokers.map(_.connectionString).mkString(",")) @@ -96,22 +96,22 @@ object ClientUtils extends Logging{ /** * Parse a list of broker urls in the form host1:port1, host2:port2, ... */ - def parseBrokerList(brokerListStr: String): Seq[Broker] = { + def parseBrokerList(brokerListStr: String): Seq[BrokerEndpoint] = { val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - new Broker(brokerId, getHost(address), getPort(address)) + BrokerEndpoint.createBrokerEndPoint(brokerId, address) } } /** * Creates a blocking channel to a random broker */ - def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = { + def channelToAnyBroker(zkClient: ZkClient, protocolType: SecurityProtocol, socketTimeoutMs: Int = 3000) : BlockingChannel = { var channel: BlockingChannel = null var connected = false while (!connected) { - val allBrokers = getAllBrokersInCluster(zkClient) + val allBrokers = getAllBrokerEndPointsForChannel(zkClient, protocolType) Random.shuffle(allBrokers).find { broker => trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) try { @@ -136,19 +136,19 @@ object ClientUtils extends Logging{ /** * Creates a blocking channel to the offset manager of the given group */ - def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { - var queryChannel = channelToAnyBroker(zkClient) + def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000, protocolType: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + var queryChannel = channelToAnyBroker(zkClient, protocolType) var offsetManagerChannelOpt: Option[BlockingChannel] = None while (!offsetManagerChannelOpt.isDefined) { - var coordinatorOpt: Option[Broker] = None + var coordinatorOpt: Option[BrokerEndpoint] = None while (!coordinatorOpt.isDefined) { try { if (!queryChannel.isConnected) - queryChannel = channelToAnyBroker(zkClient) + queryChannel = channelToAnyBroker(zkClient, protocolType) debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) queryChannel.send(ConsumerMetadataRequest(group)) val response = queryChannel.receive() diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 0060add..4ee30a9 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -18,17 +18,40 @@ package kafka.cluster import kafka.utils.Utils._ -import kafka.utils.Json -import kafka.api.ApiUtils._ +import kafka.utils.{Utils, Json} import java.nio.ByteBuffer -import kafka.common.{KafkaException, BrokerNotAvailableException} -import org.apache.kafka.common.utils.Utils._ +import kafka.common.{BrokerEndPointNotAvailableException, KafkaException, BrokerNotAvailableException} +import kafka.cluster.SecurityProtocol._ /** - * A Kafka broker + * A Kafka broker. + * A broker has an id, a host and a collection of end-points. + * Each end-point is (port,protocolType). + * Currently the only protocol type is PlainText but we will add SSL and Kerberos in the future. */ object Broker { + /** + * Create a broker object from id and JSON string. + * @param id + * @param brokerInfoString + * + * Version 1 JSON schema for a broker is: + * {"version":1, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666" } + * + * The current JSON schema for a broker is: + * {"version":2, + * "host","localhost", + * "port",9092 + * "jmx_port":9999, + * "timestamp":"2233345666", + * "endpoints": ["PLAINTEXT://host1:9092", + * "SSL://host1:9093"] + */ def createBroker(id: Int, brokerInfoString: String): Broker = { if(brokerInfoString == null) throw new BrokerNotAvailableException("Broker id %s does not exist".format(id)) @@ -36,9 +59,18 @@ object Broker { Json.parseFull(brokerInfoString) match { case Some(m) => val brokerInfo = m.asInstanceOf[Map[String, Any]] - val host = brokerInfo.get("host").get.asInstanceOf[String] - val port = brokerInfo.get("port").get.asInstanceOf[Int] - new Broker(id, host, port) + val version = brokerInfo.get("version").get.asInstanceOf[Int] + val endpoints = version match { + case 1 => + val host = brokerInfo.get("host").get.asInstanceOf[String] + val port = brokerInfo.get("port").get.asInstanceOf[Int] + Map(SecurityProtocol.PLAINTEXT -> new EndPoint(host, port, SecurityProtocol.PLAINTEXT)) + case 2 => + val listeners = brokerInfo.get("endpoints").get.asInstanceOf[List[String]] + listeners.map(listener => EndPoint.createEndPoint(listener)).map(ep => ep.protocolType -> ep).toMap + case _ => throw new KafkaException("Unknown version of broker registration. Only versions 1 and 2 are supported." + brokerInfoString) + } + new Broker(id, endpoints) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -47,36 +79,70 @@ object Broker { } } + /** + * + * @param buffer Containing serialized broker. + * Current serialization is: + * id (int), host (size + string), number of endpoints (int), serialized endpoints + * @return broker object + */ def readFrom(buffer: ByteBuffer): Broker = { val id = buffer.getInt - val host = readShortString(buffer) - val port = buffer.getInt - new Broker(id, host, port) + val numEndpoints = buffer.getInt + + val endpoints = List.range(0, numEndpoints).map(i => EndPoint.readFrom(buffer)) + .map(ep => ep.protocolType -> ep).toMap + new Broker(id, endpoints) } } -case class Broker(id: Int, host: String, port: Int) { - - override def toString: String = "id:" + id + ",host:" + host + ",port:" + port +case class Broker(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) { + + override def toString: String = id + " : " + endPoints.values.mkString("(",",",")") + + def this(id: Int, host: String, port: Int, protocol: SecurityProtocol) = { + this(id, Map(protocol -> EndPoint(host, port, protocol))) + } + + def this(bep: BrokerEndpoint, protocol: SecurityProtocol) = { + this(bep.id, bep.host, bep.port, protocol) + } - def connectionString: String = formatAddress(host, port) def writeTo(buffer: ByteBuffer) { buffer.putInt(id) - writeShortString(buffer, host) - buffer.putInt(port) + buffer.putInt(endPoints.size) + for(endpoint <- endPoints.values) { + endpoint.writeTo(buffer) + } } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + def sizeInBytes: Int = + 4 + /* broker id*/ + 4 + /* number of endPoints */ + endPoints.values.map(_.sizeInBytes).sum /* end points */ + + def supportsChannel(protocolType: SecurityProtocol): Unit = { + endPoints.contains(protocolType) + } + + def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndpoint = { + val endpoint = endPoints.get(protocolType) + endpoint match { + case Some(endpoint) => new BrokerEndpoint(id, endpoint.host, endpoint.port) + case None => + throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) + } + } override def equals(obj: Any): Boolean = { obj match { case null => false - case n: Broker => id == n.id && host == n.host && port == n.port + // Yes, Scala compares lists element by element + case n: Broker => id == n.id && endPoints == n.endPoints case _ => false } } - - override def hashCode(): Int = hashcode(id, host, port) - -} + + override def hashCode(): Int = hashcode(id, endPoints) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9ebbee6..4112343 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,6 +19,7 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest +import kafka.cluster.SecurityProtocol import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -180,6 +181,12 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) + + /** + * Only PLAINTEXT protocol is supported on the scala consumer, so no need to set this + * This parameter is used for testing. + **/ + val securityProtocol = SecurityProtocol.withName(props.getString("security.protocol", "PLAINTEXT")) validate(this) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index b9e2bea..59e9876 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -19,9 +19,8 @@ package kafka.consumer import org.I0Itec.zkclient.ZkClient import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} -import kafka.cluster.{Cluster, Broker} +import kafka.cluster.{BrokerEndpoint, Cluster} import scala.collection.immutable -import scala.collection.Map import collection.mutable.HashMap import scala.collection.mutable import java.util.concurrent.locks.ReentrantLock @@ -53,7 +52,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { - val leaderForPartitionsMap = new HashMap[TopicAndPartition, Broker] + val leaderForPartitionsMap = new HashMap[TopicAndPartition, BrokerEndpoint] lock.lock() try { while (noLeaderPartitionSet.isEmpty) { @@ -62,7 +61,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokersInCluster(zkClient) + val brokers = getAllBrokerEndPointsForChannel(zkClient, config.securityProtocol) val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.clientId, @@ -114,7 +113,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, } } - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread = { new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), config, sourceBroker, partitionMap, this) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 152fda5..cde4481 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.consumer -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.server.AbstractFetcherThread import kafka.message.ByteBufferMessageSet import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData} @@ -26,7 +26,7 @@ import kafka.common.TopicAndPartition class ConsumerFetcherThread(name: String, val config: ConsumerConfig, - sourceBroker: Broker, + sourceBroker: BrokerEndpoint, partitionMap: Map[TopicAndPartition, PartitionTopicInfo], val consumerFetcherManager: ConsumerFetcherManager) extends AbstractFetcherThread(name = name, diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index cca815a..46b3e20 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -185,7 +185,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def ensureOffsetManagerConnected() { if (config.offsetsStorage == "kafka") { if (offsetsChannel == null || !offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, + config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs, config.securityProtocol) debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index c582191..0fc02b7 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -18,6 +18,7 @@ package kafka.controller import kafka.network.{Receive, BlockingChannel} import kafka.utils.{Utils, Logging, ShutdownableThread} +import org.apache.kafka.common.protocol.ApiVersion import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} @@ -80,7 +81,8 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val channel = new BlockingChannel(broker.host, broker.port, + val brokerEndPoint = broker.getBrokerEndPoint(config.intraBrokerSecurityProtocol) + val channel = new BlockingChannel(brokerEndPoint.host, brokerEndPoint.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) @@ -284,7 +286,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map(b => b.getBrokerEndPoint(controller.config.intraBrokerSecurityProtocol)) val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfos, leaders, controllerId, controllerEpoch, correlationId, clientId) for (p <- partitionStateInfos) { val typeOfRequest = if (broker == p._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader" else "become-follower" @@ -299,8 +301,10 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging updateMetadataRequestMap.foreach { m => val broker = m._1 val partitionStateInfos = m._2.toMap - val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + + val versionId = if (controller.config.intraBrokerProtocolVersion.onOrAfter(ApiVersion.KAFKA_083)) 1 else 0 + val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, + correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1))) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 09fc46d..789cea4 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -212,7 +212,11 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt def epoch = controllerContext.epoch - def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port) + def clientId = { + val listeners = config.listeners + val controllerListener = listeners.get(config.intraBrokerSecurityProtocol) + "id_%d-host_%s-port_%d".format(config.brokerId, controllerListener.get.host, controllerListener.get.port) + } /** * On clean shutdown, the controller first determines the partitions that the diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala index d281bb3..9c14428 100644 --- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -18,14 +18,13 @@ package kafka.javaapi import java.nio.ByteBuffer - -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { def errorCode = underlying.errorCode - def coordinator: Broker = { + def coordinator: BrokerEndpoint = { import kafka.javaapi.Implicits._ underlying.coordinatorOpt } diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala index f384e04..ebbd589 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -16,7 +16,7 @@ */ package kafka.javaapi -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import scala.collection.JavaConversions private[javaapi] object MetadataListImplicits { @@ -52,17 +52,17 @@ class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { def partitionId: Int = underlying.partitionId - def leader: Broker = { + def leader: BrokerEndpoint = { import kafka.javaapi.Implicits._ underlying.leader } - def replicas: java.util.List[Broker] = { + def replicas: java.util.List[BrokerEndpoint] = { import JavaConversions._ underlying.replicas } - def isr: java.util.List[Broker] = { + def isr: java.util.List[BrokerEndpoint] = { import JavaConversions._ underlying.isr } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 7b1db3d..c4bad46 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -18,6 +18,8 @@ package kafka.network import java.util.concurrent._ +import kafka.cluster.SecurityProtocol +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge import java.nio.ByteBuffer @@ -30,7 +32,7 @@ import org.apache.log4j.Logger object RequestChannel extends Logging { - val AllDone = new Request(1, 2, getShutdownReceive(), 0) + val AllDone = new Request(processor = 1, requestKey = 2, buffer = getShutdownReceive(), startTimeMs = 0, securityProtocol = SecurityProtocol.PLAINTEXT) def getShutdownReceive() = { val emptyProducerRequest = new ProducerRequest(0, 0, "", 0, 0, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]()) @@ -41,7 +43,7 @@ object RequestChannel extends Logging { byteBuffer } - case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0)) { + case class Request(processor: Int, requestKey: Any, private var buffer: ByteBuffer, startTimeMs: Long, remoteAddress: SocketAddress = new InetSocketAddress(0), securityProtocol: SecurityProtocol) { @volatile var requestDequeueTimeMs = -1L @volatile var apiLocalCompleteTimeMs = -1L @volatile var responseCompleteTimeMs = -1L diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 76ce41a..7069da0 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -24,7 +24,11 @@ import java.net._ import java.io._ import java.nio.channels._ +import kafka.cluster.EndPoint +import kafka.cluster.SecurityProtocol.SecurityProtocol + import scala.collection._ +import scala.collection.JavaConversions._ import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup @@ -38,8 +42,7 @@ import com.yammer.metrics.core.{Gauge, Meter} * M Handler threads that handle requests and produce responses back to the processor threads for writing. */ class SocketServer(val brokerId: Int, - val host: String, - val port: Int, + val endpoints: Map[SecurityProtocol, EndPoint], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, @@ -51,28 +54,39 @@ class SocketServer(val brokerId: Int, this.logIdent = "[Socket Server on Broker " + brokerId + "], " private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) - @volatile private var acceptor: Acceptor = null + @volatile private[network] var acceptors: ConcurrentHashMap[EndPoint,Acceptor] = new ConcurrentHashMap[EndPoint,Acceptor]() val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) /* a meter to track the average free capacity of the network processors */ private val aggregateIdleMeter = newMeter("NetworkProcessorAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) + + /* I'm pushing the mapping of port-to-protocol to the processor level, + so the processor can put the correct protocol in the request channel. + we'll probably have a more elegant way of doing this once we patch the request channel + to include more information about security and authentication. + TODO: re-consider this code when working on KAFKA-1683 + */ + private val portToProtocol: Map[Int, SecurityProtocol] = + endpoints.map{ case (protocol: SecurityProtocol, endpoint: EndPoint) => (endpoint.port -> protocol )} + /** * Start the socket server */ def startup() { val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(i, - time, - maxRequestSize, + processors(i) = new Processor(i, + time, + maxRequestSize, aggregateIdleMeter, newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), - numProcessorThreads, + numProcessorThreads, requestChannel, quotas, - connectionsMaxIdleMs) - Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start() + connectionsMaxIdleMs, + portToProtocol) + Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() } newGauge("ResponsesBeingSent", new Gauge[Int] { @@ -83,10 +97,17 @@ class SocketServer(val brokerId: Int, requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) // start accepting connections - this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize, quotas) - Utils.newThread("kafka-socket-acceptor", acceptor, false).start() - acceptor.awaitStartup - info("Started") + // right now we will use the same processors for all ports, since we didn't implement different protocols + // in the future, we may implement different processors for SSL and Kerberos + + endpoints.values.foreach(endpoint => { + val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas) + acceptors.put(endpoint,acceptor) + Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() + acceptor.awaitStartup + }) + + info("Started " + acceptors.size() + " acceptor threads") } /** @@ -94,8 +115,8 @@ class SocketServer(val brokerId: Int, */ def shutdown() = { info("Shutting down") - if(acceptor != null) - acceptor.shutdown() + if(acceptors != null) + acceptors.values().foreach(_.shutdown()) for(processor <- processors) processor.shutdown() info("Shutdown completed") @@ -301,7 +322,8 @@ private[kafka] class Processor(val id: Int, val totalProcessorThreads: Int, val requestChannel: RequestChannel, connectionQuotas: ConnectionQuotas, - val connectionsMaxIdleMs: Long) extends AbstractServerThread(connectionQuotas) { + val connectionsMaxIdleMs: Long, + val portToProtocol: Map[Int,SecurityProtocol]) extends AbstractServerThread(connectionQuotas) { private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() private val connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000 @@ -447,7 +469,9 @@ private[kafka] class Processor(val id: Int, if(read < 0) { close(key) } else if(receive.complete) { - val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address) + val port = socketChannel.socket().getLocalPort + val protocol = portToProtocol(port) + val req = RequestChannel.Request(processor = id, requestKey = key, buffer = receive.buffer, startTimeMs = time.milliseconds, remoteAddress = address, securityProtocol = protocol) requestChannel.sendRequest(req) key.attach(null) // explicitly reset interest ops to not READ, no need to wake up the selector just yet diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 43df70b..362623f 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -17,7 +17,7 @@ package kafka.producer -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import java.util.Properties import collection.mutable.HashMap import java.lang.Object @@ -30,7 +30,7 @@ object ProducerPool { /** * Used in ProducerPool to initiate a SyncProducer connection with a broker. */ - def createSyncProducer(config: ProducerConfig, broker: Broker): SyncProducer = { + def createSyncProducer(config: ProducerConfig, broker: BrokerEndpoint): SyncProducer = { val props = new Properties() props.put("host", broker.host) props.put("port", broker.port.toString) @@ -44,11 +44,12 @@ class ProducerPool(val config: ProducerConfig) extends Logging { private val lock = new Object() def updateProducer(topicMetadata: Seq[TopicMetadata]) { - val newBrokers = new collection.mutable.HashSet[Broker] + val newBrokers = new collection.mutable.HashSet[BrokerEndpoint] topicMetadata.foreach(tmd => { tmd.partitionsMetadata.foreach(pmd => { - if(pmd.leader.isDefined) - newBrokers+=(pmd.leader.get) + if(pmd.leader.isDefined) { + newBrokers += pmd.leader.get + } }) }) lock synchronized { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 20c00cb..94aa952 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.collection.Set import scala.collection.Map import kafka.utils.{Utils, Logging} -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge @@ -68,7 +68,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri } // to be defined in subclass to create a specific fetcher - def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread + def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread def addFetcherForPartitions(partitionAndOffsets: Map[TopicAndPartition, BrokerAndInitialOffset]) { mapLock synchronized { @@ -126,6 +126,6 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri } } -case class BrokerAndFetcherId(broker: Broker, fetcherId: Int) +case class BrokerAndFetcherId(broker: BrokerEndpoint, fetcherId: Int) -case class BrokerAndInitialOffset(broker: Broker, initOffset: Long) \ No newline at end of file +case class BrokerAndInitialOffset(broker: BrokerEndpoint, initOffset: Long) \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index e731df4..748c328 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.utils.{Pool, ShutdownableThread} import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} @@ -36,7 +36,8 @@ import com.yammer.metrics.core.Gauge /** * Abstract class for fetching data from multiple partitions from the same broker. */ -abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int, + +abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: BrokerEndpoint, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, fetchBackOffMs: Int = 0, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 35af98f..eea6011 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -19,8 +19,8 @@ package kafka.server import org.apache.kafka.common.requests.JoinGroupResponse import org.apache.kafka.common.requests.HeartbeatResponse +import kafka.cluster.SecurityProtocol.SecurityProtocol import org.apache.kafka.common.TopicPartition - import kafka.api._ import kafka.admin.AdminUtils import kafka.common._ @@ -350,8 +350,8 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - private def getTopicMetadata(topics: Set[String]): Seq[TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics) + private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol) if (topics.size > 0 && topicResponses.size != topics.size) { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => @@ -393,10 +393,10 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet, request.securityProtocol) val brokers = metadataCache.getAliveBrokers trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId) + val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } @@ -433,7 +433,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = offsetManager.partitionFor(consumerMetadataRequest.group) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName)).head + val offsetsTopicMetadata = getTopicMetadata(Set(OffsetManager.OffsetsTopicName), request.securityProtocol).head val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 46d21c7..2443244 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,12 +19,17 @@ package kafka.server import java.util.Properties +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{SecurityProtocol, EndPoint} +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.common.InvalidConfigException import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.Utils import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.protocol.ApiVersion -import scala.collection.{JavaConversions, Map} +import scala.collection.{immutable, JavaConversions, Map} object Defaults { /** ********* Zookeeper Configuration ***********/ @@ -102,6 +107,8 @@ object Defaults { val LeaderImbalancePerBrokerPercentage = 10 val LeaderImbalanceCheckIntervalSeconds = 300 val UncleanLeaderElectionEnable = true + val IntraBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString + val IntraBrokerProtocolVersion = ApiVersion.getLatestVersion.toString /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetries = 3 @@ -143,8 +150,10 @@ object KafkaConfig { /** ********* Socket Server Configuration ***********/ val PortProp = "port" val HostNameProp = "host.name" + val ListenersProp = "listeners" val AdvertisedHostNameProp: String = "advertised.host.name" val AdvertisedPortProp = "advertised.port" + val AdvertisedListenersProp = "advertised.listeners" val SocketSendBufferBytesProp = "socket.send.buffer.bytes" val SocketReceiveBufferBytesProp = "socket.receive.buffer.bytes" val SocketRequestMaxBytesProp = "socket.request.max.bytes" @@ -209,6 +218,8 @@ object KafkaConfig { val LeaderImbalancePerBrokerPercentageProp = "leader.imbalance.per.broker.percentage" val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds" val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" + val IntraBrokerSecurityProtocolProp = "security.intra.broker.protocol" + val IntraBrokerProtocolVersionProp = "use.intra.broker.protocol.version" /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" @@ -248,6 +259,12 @@ object KafkaConfig { /** ********* Socket Server Configuration ***********/ val PortDoc = "the port to listen and accept connections on" val HostNameDoc = "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" + val ListenersDoc = "Listener List - Comma-separated list of URIs we will listen on and their protocols.\n" + + " Specify hostname as 0.0.0.0 to bind to all interfaces.\n" + + " Leave hostname empty to bind to default interface.\n" + + " Examples of legal listener lists:\n" + + " PLAINTEXT://myhost:9092,TRACE://:9091\n" + + " PLAINTEXT://0.0.0.0:9092, TRACE://localhost:9093\n" val AdvertisedHostNameDoc = "Hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may " + "need to be different from the interface to which the broker binds. If this is not set, " + "it will use the value for \"host.name\" if configured. Otherwise " + @@ -255,6 +272,9 @@ object KafkaConfig { val AdvertisedPortDoc = "The port to publish to ZooKeeper for clients to use. In IaaS environments, this may " + "need to be different from the port to which the broker binds. If this is not set, " + "it will publish the same port that the broker binds to." + val AdvertisedListenersDoc = "Listeners to publish to ZooKeeper for clients to use, if different than the listeners above." + + " In IaaS environments, this may need to be different from the interface to which the broker binds." + + " If this is not set, the value for \"listeners\" will be used." val SocketSendBufferBytesDoc = "The SO_SNDBUF buffer of the socket sever sockets" val SocketReceiveBufferBytesDoc = "The SO_RCVBUF buffer of the socket sever sockets" val SocketRequestMaxBytesDoc = "The maximum number of bytes in a socket request" @@ -321,6 +341,10 @@ object KafkaConfig { val LeaderImbalancePerBrokerPercentageDoc = "The ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above this value per broker. The value is specified in percentage." val LeaderImbalanceCheckIntervalSecondsDoc = "The frequency with which the partition rebalance check is triggered by the controller" val UncleanLeaderElectionEnableDoc = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss" + val IntraBrokerSecurityProtocolDoc = "Security protocol used to communicate between brokers. Defaults to plain text." + val IntraBrokerProtocolVersionDoc = "Specify which version of the inter-broker protocol will be used.\n" + + " This is typically bumped after all brokers were upgraded to a new version.\n" + + " Example of some valid values are: 0.8.2.0, 0.8.2.1, 0.8.3.0. Check ApiVersion for the full list." /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." @@ -352,7 +376,6 @@ object KafkaConfig { import ConfigDef.ValidString._ import ConfigDef.Type._ import ConfigDef.Importance._ - import java.util.Arrays.asList new ConfigDef() @@ -374,8 +397,10 @@ object KafkaConfig { /** ********* Socket Server Configuration ***********/ .define(PortProp, INT, Defaults.Port, HIGH, PortDoc) .define(HostNameProp, STRING, Defaults.HostName, HIGH, HostNameDoc) + .define(ListenersProp, STRING, HIGH, ListenersDoc, false) .define(AdvertisedHostNameProp, STRING, HIGH, AdvertisedHostNameDoc, false) .define(AdvertisedPortProp, INT, HIGH, AdvertisedPortDoc, false) + .define(AdvertisedListenersProp, STRING, HIGH, AdvertisedListenersDoc, false) .define(SocketSendBufferBytesProp, INT, Defaults.SocketSendBufferBytes, HIGH, SocketSendBufferBytesDoc) .define(SocketReceiveBufferBytesProp, INT, Defaults.SocketReceiveBufferBytes, HIGH, SocketReceiveBufferBytesDoc) .define(SocketRequestMaxBytesProp, INT, Defaults.SocketRequestMaxBytes, atLeast(1), HIGH, SocketRequestMaxBytesDoc) @@ -442,7 +467,8 @@ object KafkaConfig { .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) .define(LeaderImbalanceCheckIntervalSecondsProp, INT, Defaults.LeaderImbalanceCheckIntervalSeconds, HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, HIGH, UncleanLeaderElectionEnableDoc) - + .define(IntraBrokerSecurityProtocolProp, STRING, Defaults.IntraBrokerSecurityProtocol, MEDIUM, IntraBrokerSecurityProtocolDoc) + .define(IntraBrokerProtocolVersionProp, STRING, Defaults.IntraBrokerProtocolVersion, MEDIUM, IntraBrokerProtocolVersionDoc) /** ********* Controlled shutdown configuration ***********/ .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) .define(ControlledShutdownRetryBackoffMsProp, INT, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) @@ -493,8 +519,10 @@ object KafkaConfig { /** ********* Socket Server Configuration ***********/ port = parsed.get(PortProp).asInstanceOf[Int], hostName = parsed.get(HostNameProp).asInstanceOf[String], + _listeners = Option(parsed.get(ListenersProp)).map(_.asInstanceOf[String]), _advertisedHostName = Option(parsed.get(AdvertisedHostNameProp)).map(_.asInstanceOf[String]), _advertisedPort = Option(parsed.get(AdvertisedPortProp)).map(_.asInstanceOf[Int]), + _advertisedListeners = Option(parsed.get(AdvertisedListenersProp)).map(_.asInstanceOf[String]), socketSendBufferBytes = parsed.get(SocketSendBufferBytesProp).asInstanceOf[Int], socketReceiveBufferBytes = parsed.get(SocketReceiveBufferBytesProp).asInstanceOf[Int], socketRequestMaxBytes = parsed.get(SocketRequestMaxBytesProp).asInstanceOf[Int], @@ -561,7 +589,8 @@ object KafkaConfig { leaderImbalancePerBrokerPercentage = parsed.get(LeaderImbalancePerBrokerPercentageProp).asInstanceOf[Int], leaderImbalanceCheckIntervalSeconds = parsed.get(LeaderImbalanceCheckIntervalSecondsProp).asInstanceOf[Int], uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], - + intraBrokerSecurityProtocol = SecurityProtocol.withName(parsed.get(IntraBrokerSecurityProtocolProp).asInstanceOf[String]), + intraBrokerProtocolVersion = ApiVersion.parseConfig(parsed.get(IntraBrokerProtocolVersionProp).asInstanceOf[String]), /** ********* Controlled shutdown configuration ***********/ controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], @@ -632,8 +661,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ /** ********* Socket Server Configuration ***********/ val port: Int = Defaults.Port, val hostName: String = Defaults.HostName, + private val _listeners: Option[String] = None, private val _advertisedHostName: Option[String] = None, private val _advertisedPort: Option[Int] = None, + private val _advertisedListeners: Option[String] = None, val socketSendBufferBytes: Int = Defaults.SocketSendBufferBytes, val socketReceiveBufferBytes: Int = Defaults.SocketReceiveBufferBytes, val socketRequestMaxBytes: Int = Defaults.SocketRequestMaxBytes, @@ -702,6 +733,8 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val leaderImbalancePerBrokerPercentage: Int = Defaults.LeaderImbalancePerBrokerPercentage, val leaderImbalanceCheckIntervalSeconds: Int = Defaults.LeaderImbalanceCheckIntervalSeconds, val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, + val intraBrokerSecurityProtocol: SecurityProtocol = SecurityProtocol.withName(Defaults.IntraBrokerSecurityProtocol), + val intraBrokerProtocolVersion: ApiVersion = ApiVersion.parseConfig(Defaults.IntraBrokerProtocolVersion), /** ********* Controlled shutdown configuration ***********/ val controlledShutdownMaxRetries: Int = Defaults.ControlledShutdownMaxRetries, @@ -726,8 +759,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) + val listeners = getListeners() val advertisedHostName: String = _advertisedHostName.getOrElse(hostName) val advertisedPort: Int = _advertisedPort.getOrElse(port) + val advertisedListeners = getAdvertisedListeners() val logDirs = Utils.parseCsvList(_logDirs.getOrElse(_logDir)) val logRollTimeMillis = _logRollTimeMillis.getOrElse(60 * 60 * 1000L * logRollTimeHours) @@ -736,14 +771,6 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val logFlushIntervalMs = _logFlushIntervalMs.getOrElse(logFlushSchedulerIntervalMs) - private def getMap(propName: String, propValue: String): Map[String, String] = { - try { - Utils.parseCsvMap(propValue) - } catch { - case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage)) - } - } - val maxConnectionsPerIpOverrides: Map[String, Int] = getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, _maxConnectionsPerIpOverrides).map { case (k, v) => (k, v.toInt)} @@ -759,6 +786,56 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ ) } + private def getMap(propName: String, propValue: String): Map[String, String] = { + try { + Utils.parseCsvMap(propValue) + } catch { + case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage)) + } + } + + def validateUniquePortAndProtocol(listeners: String) { + + val endpoints = try { + val listenerList = Utils.parseCsvList(listeners) + listenerList.map(listener => EndPoint.createEndPoint(listener)) + } catch { + case e: Exception => throw new IllegalArgumentException("Error creating broker listeners from '%s': %s".format(listeners, e.getMessage)) + } + val distinctPorts = endpoints.map(ep => ep.port).distinct + val distinctProtocols = endpoints.map(ep => ep.protocolType).distinct + + require(distinctPorts.size == endpoints.size) + require(distinctProtocols.size == endpoints.size) + } + + // If the user did not define listeners but did define host or port, let's use them in backward compatible way + // If none of those are defined, we default to PLAINTEXT://null:6667 + private def getListeners(): immutable.Map[SecurityProtocol, EndPoint] = { + if (_listeners.isDefined) { + validateUniquePortAndProtocol(_listeners.get) + Utils.listenerListToEndPoints(_listeners.get) + } else { + Utils.listenerListToEndPoints("PLAINTEXT://" + hostName + ":" + port) + } + } + + // If the user defined advertised listeners, we use those + // If he didn't but did define advertised host or port, we'll use those and fill in the missing value from regular host / port or defaults + // If none of these are defined, we'll use the listeners + private def getAdvertisedListeners(): immutable.Map[SecurityProtocol, EndPoint] = { + if (_advertisedListeners.isDefined) { + validateUniquePortAndProtocol(_advertisedListeners.get) + Utils.listenerListToEndPoints(_advertisedListeners.get) + } else if (_advertisedHostName.isDefined || _advertisedPort.isDefined ) { + Utils.listenerListToEndPoints("PLAINTEXT://" + advertisedHostName + ":" + advertisedPort) + } else { + getListeners() + } + } + + + validateValues() private def validateValues() { @@ -802,8 +879,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ /** ********* Socket Server Configuration ***********/ props.put(PortProp, port.toString) props.put(HostNameProp, hostName) + _listeners.foreach(props.put(ListenersProp, _)) _advertisedHostName.foreach(props.put(AdvertisedHostNameProp, _)) _advertisedPort.foreach(value => props.put(AdvertisedPortProp, value.toString)) + _advertisedListeners.foreach(props.put(AdvertisedListenersProp, _)) props.put(SocketSendBufferBytesProp, socketSendBufferBytes.toString) props.put(SocketReceiveBufferBytesProp, socketReceiveBufferBytes.toString) props.put(SocketRequestMaxBytesProp, socketRequestMaxBytes.toString) @@ -871,6 +950,9 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(LeaderImbalancePerBrokerPercentageProp, leaderImbalancePerBrokerPercentage.toString) props.put(LeaderImbalanceCheckIntervalSecondsProp, leaderImbalanceCheckIntervalSeconds.toString) props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) + props.put(IntraBrokerSecurityProtocolProp, intraBrokerSecurityProtocol.toString) + props.put(IntraBrokerProtocolVersionProp, intraBrokerProtocolVersion.toString) + /** ********* Controlled shutdown configuration ***********/ props.put(ControlledShutdownMaxRetriesProp, controlledShutdownMaxRetries.toString) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 7907987..7a1378e 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,6 +17,8 @@ package kafka.server +import kafka.cluster.{SecurityProtocol, EndPoint} +import kafka.cluster.SecurityProtocol.SecurityProtocol import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -31,9 +33,8 @@ import java.net.InetAddress * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. */ -class KafkaHealthcheck(private val brokerId: Int, - private val advertisedHost: String, - private val advertisedPort: Int, +class KafkaHealthcheck(private val brokerId: Int, + private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { @@ -49,13 +50,18 @@ class KafkaHealthcheck(private val brokerId: Int, * Register this broker as "alive" in zookeeper */ def register() { - val advertisedHostName = - if(advertisedHost == null || advertisedHost.trim.isEmpty) - InetAddress.getLocalHost.getCanonicalHostName - else - advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, zkSessionTimeoutMs, jmxPort) + val updatedEndpoints = advertisedEndpoints.mapValues(endpoint => + if (endpoint.host == null || endpoint.host.trim.isEmpty) + EndPoint(InetAddress.getLocalHost.getCanonicalHostName, endpoint.port, endpoint.protocolType) + else + endpoint + ) + + // the default host and port are here for compatibility with older client + // only PLAINTEXT is supported as default + val defaultEndpoint = updatedEndpoints(SecurityProtocol.PLAINTEXT) + ZkUtils.registerBrokerInZk(zkClient, brokerId, defaultEndpoint.host, defaultEndpoint.port, updatedEndpoints, zkSessionTimeoutMs, jmxPort) } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dddef93..cda2e01 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -25,6 +25,7 @@ import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File + import collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} @@ -117,56 +118,55 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg this.logIdent = "[Kafka Server " + config.brokerId + "], " socketServer = new SocketServer(config.brokerId, - config.hostName, - config.port, - config.numNetworkThreads, - config.queuedMaxRequests, - config.socketSendBufferBytes, - config.socketReceiveBufferBytes, - config.socketRequestMaxBytes, - config.maxConnectionsPerIp, - config.connectionsMaxIdleMs, - config.maxConnectionsPerIpOverrides) - socketServer.startup() - - /* start replica manager */ - replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) - replicaManager.startup() - - /* start offset manager */ - offsetManager = createOffsetManager() - - /* start kafka controller */ - kafkaController = new KafkaController(config, zkClient, brokerState) - kafkaController.startup() - - /* start kafka coordinator */ - consumerCoordinator = new ConsumerCoordinator(config, zkClient) - consumerCoordinator.startup() - - /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache) - requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - brokerState.newState(RunningAsBroker) - - Mx4jLoader.maybeLoad() - - /* start topic config manager */ - topicConfigManager = new TopicConfigManager(zkClient, logManager) - topicConfigManager.startup() - - /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) - kafkaHealthcheck.startup() - - /* register broker metrics */ - registerStats() - - shutdownLatch = new CountDownLatch(1) - startupComplete.set(true) - isStartingUp.set(false) - info("started") + config.listeners, + config.numNetworkThreads, + config.queuedMaxRequests, + config.socketSendBufferBytes, + config.socketReceiveBufferBytes, + config.socketRequestMaxBytes, + config.maxConnectionsPerIp, + config.connectionsMaxIdleMs, + config.maxConnectionsPerIpOverrides) + socketServer.startup() + + /* start replica manager */ + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + replicaManager.startup() + + /* start offset manager */ + offsetManager = createOffsetManager() + + /* start kafka controller */ + kafkaController = new KafkaController(config, zkClient, brokerState) + kafkaController.startup() + + /* start kafka coordinator */ + consumerCoordinator = new ConsumerCoordinator(config, zkClient) + consumerCoordinator.startup() + + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache) + requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) + brokerState.newState(RunningAsBroker) + + Mx4jLoader.maybeLoad() + + /* start topic config manager */ + topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager.startup() + + /* tell everyone we are alive */ + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedListeners, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck.startup() + + /* register broker metrics */ + registerStats() + + shutdownLatch = new CountDownLatch(1) + startupComplete.set(true) + isStartingUp.set(false) + info("started") } } catch { @@ -242,7 +242,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.host, broker.port, + channel = new BlockingChannel(broker.getBrokerEndPoint(config.intraBrokerSecurityProtocol).host, + broker.getBrokerEndPoint(config.intraBrokerSecurityProtocol).port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) @@ -417,7 +418,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg *
  • config has broker.id and meta.properties contains broker.id if they don't match throws InconsistentBrokerIdException *
  • config has broker.id and there is no meta.properties file, creates new meta.properties and stores broker.id *
      - * @returns A brokerId. + * @return A brokerId. */ private def getBrokerId: Int = { var brokerId = config.brokerId diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 6aef6e4..675e9c9 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,9 +17,12 @@ package kafka.server +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{BrokerEndpoint,Broker} +import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} +import kafka.common.TopicAndPartition + import kafka.api._ -import kafka.common._ -import kafka.cluster.Broker import kafka.controller.KafkaController.StateChangeLogger import scala.collection.{Seq, Set, mutable} import kafka.utils.Logging @@ -39,7 +42,8 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { this.logIdent = "[Kafka Metadata Cache on broker %d] ".format(brokerId) - def getTopicMetadata(topics: Set[String]) = { + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol) = { + val isAllTopics = topics.isEmpty val topicsRequested = if(isAllTopics) cache.keySet else topics val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] @@ -50,18 +54,21 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { val partitionMetadata = partitionStateInfos.map { case (partitionId, partitionState) => val replicas = partitionState.allReplicas - val replicaInfo: Seq[Broker] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq - var leaderInfo: Option[Broker] = None - var isrInfo: Seq[Broker] = Nil + val replicaInfo: Seq[BrokerEndpoint] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq.map(_.getBrokerEndPoint(protocol)) + var leaderInfo: Option[BrokerEndpoint] = None + var leaderBrokerInfo: Option[Broker] = None + var isrInfo: Seq[BrokerEndpoint] = Nil val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch val leader = leaderIsrAndEpoch.leaderAndIsr.leader val isr = leaderIsrAndEpoch.leaderAndIsr.isr val topicPartition = TopicAndPartition(topic, partitionId) try { - leaderInfo = aliveBrokers.get(leader) - if (!leaderInfo.isDefined) + leaderBrokerInfo = aliveBrokers.get(leader) + if (!leaderBrokerInfo.isDefined) throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null) + else + leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) + isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).map(_.getBrokerEndPoint(protocol)) if (replicaInfo.size < replicas.size) throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 351dbba..f0a2a5b 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -17,13 +17,13 @@ package kafka.server -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager) extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId, "Replica", brokerConfig.numReplicaFetchers) { - override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = { + override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndpoint): AbstractFetcherThread = { new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 96faa7b..b2196c8 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -18,14 +18,14 @@ package kafka.server import kafka.admin.AdminUtils -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet import kafka.api.{OffsetRequest, FetchResponsePartitionData} import kafka.common.{KafkaStorageException, TopicAndPartition} class ReplicaFetcherThread(name:String, - sourceBroker: Broker, + sourceBroker: BrokerEndpoint, brokerConfig: KafkaConfig, replicaMgr: ReplicaManager) extends AbstractFetcherThread(name = name, diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index c527482..f8a118a 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.api._ import kafka.common._ import kafka.utils._ -import kafka.cluster.{Broker, Partition, Replica} +import kafka.cluster.{BrokerEndpoint, Partition, Replica} import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController @@ -652,7 +652,7 @@ class ReplicaManager(val config: KafkaConfig, * the error message will be set on each partition since we do not know which partition caused it */ private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], - leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], + leaders: Set[BrokerEndpoint], correlationId: Int, responseMap: mutable.Map[(String, Int), Short], offsetManager: OffsetManager) { partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index d1e7c43..03b121d 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -19,6 +19,7 @@ package kafka.tools import joptsimple._ +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.utils._ import kafka.consumer.SimpleConsumer @@ -158,7 +159,7 @@ object ConsumerOffsetChecker extends Logging { topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq - val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs, SecurityProtocol.PLAINTEXT) debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) channel.send(OffsetFetchRequest(group, topicPartitions)) diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index ba6ddd7..d1050b4 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -18,7 +18,7 @@ package kafka.tools import joptsimple.OptionParser -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import kafka.message.{MessageSet, MessageAndOffset, ByteBufferMessageSet} import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicReference @@ -197,7 +197,7 @@ private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, c private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], leadersPerBroker: Map[Int, Seq[TopicAndPartition]], expectedNumFetchers: Int, - brokerMap: Map[Int, Broker], + brokerMap: Map[Int, BrokerEndpoint], initialOffsetTime: Long, reportInterval: Long) extends Logging { private val fetchOffsetMap = new Pool[TopicAndPartition, Long] @@ -335,7 +335,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } } -private class ReplicaFetcher(name: String, sourceBroker: Broker, topicAndPartitions: Iterable[TopicAndPartition], +private class ReplicaFetcher(name: String, sourceBroker: BrokerEndpoint, topicAndPartitions: Iterable[TopicAndPartition], replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) extends ShutdownableThread(name) { diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index b4f903b..7379fe3 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,7 @@ import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils import kafka.api.{OffsetRequest, FetchRequestBuilder, Request} -import kafka.cluster.Broker +import kafka.cluster.BrokerEndpoint import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition @@ -142,8 +142,8 @@ object SimpleConsumerShell extends Logging { } // validating replica id and initializing target broker - var fetchTargetBroker: Broker = null - var replicaOpt: Option[Broker] = null + var fetchTargetBroker: BrokerEndpoint = null + var replicaOpt: Option[BrokerEndpoint] = null if(replicaId == UseLeaderReplica) { replicaOpt = partitionMetadataOpt.get.leader if(!replicaOpt.isDefined) { @@ -167,7 +167,9 @@ object SimpleConsumerShell extends Logging { System.exit(1) } if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout, + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, + ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, clientId) try { startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, @@ -188,8 +190,12 @@ object SimpleConsumerShell extends Logging { val replicaString = if(replicaId > 0) "leader" else "replica" info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" - .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId) + .format(topic, partitionId, replicaString, replicaId, + fetchTargetBroker.host, + fetchTargetBroker.port, startingOffset)) + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, + fetchTargetBroker.port, + 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { var offset = startingOffset diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 111c9a8..c6c5a88 100644 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -17,6 +17,7 @@ package kafka.tools +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} @@ -65,7 +66,9 @@ object UpdateOffsetsInZK { ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk") + val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host, + brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port, + 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 738c1af..1ba2597 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -24,9 +24,12 @@ import java.nio.channels._ import java.util.concurrent.locks.{ReadWriteLock, Lock} import java.lang.management._ import javax.management._ + import scala.collection._ import scala.collection.mutable import java.util.Properties +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.EndPoint import kafka.common.KafkaException import kafka.common.KafkaStorageException @@ -607,4 +610,9 @@ object Utils extends Logging { .filter{ case (k,l) => (l > 1) } .keys } + + def listenerListToEndPoints(listeners: String): immutable.Map[SecurityProtocol, EndPoint] = { + val listenerList = parseCsvList(listeners) + listenerList.map(listener => EndPoint.createEndPoint(listener)).map(ep => ep.protocolType -> ep).toMap + } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 7ae999e..6f0294f 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,7 +17,8 @@ package kafka.utils -import kafka.cluster.{Broker, Cluster} +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster._ import kafka.consumer.{ConsumerThreadId, TopicCount} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, @@ -84,6 +85,10 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } + def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: SecurityProtocol): Seq[BrokerEndpoint] = { + getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType)) + } + def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } @@ -169,12 +174,28 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + /** + * Register brokers with v2 json format (which includes multiple endpoints). + * This format also includes default endpoints for compatibility with older clients. + * @param zkClient + * @param id + * @param advertisedEndpoints + * @param timeout + * @param jmxPort + */ + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, host, port) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.toString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, advertisedEndpoints) + + registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) + + info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) + } + + def registerBrokerInZk(zkClient: ZkClient, brokerIdPath: String, brokerInfo: String, expectedBroker: Broker, timeout: Int) { try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, (brokerString: String, broker: Any) => Broker.createBroker(broker.asInstanceOf[Broker].id, brokerString).equals(broker.asInstanceOf[Broker]), @@ -183,11 +204,10 @@ object ZkUtils extends Logging { } catch { case e: ZkNodeExistsException => throw new RuntimeException("A broker is already registered on the path " + brokerIdPath - + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " - + "else you have shutdown this broker and restarted it faster than the zookeeper " - + "timeout so it appears to be re-registering.") + + ". This probably " + "indicates that you either have configured a brokerid that is already in use, or " + + "else you have shutdown this broker and restarted it faster than the zookeeper " + + "timeout so it appears to be re-registering.") } - info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) } def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 7eb6d05..518fd41 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -19,8 +19,6 @@ package kafka.api import org.junit.Test import org.junit.Assert._ - -import java.lang.Integer import java.util.{Properties, Random} import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} @@ -68,8 +66,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") + + val endpoint1 = configs(0).listeners.values.head + val endpoint2 = configs(1).listeners.values.head + consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize) producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) @@ -267,7 +268,6 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { server.shutdown() server.awaitShutdown() server.startup() - Thread.sleep(2000) } diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 3df4507..5f38557 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -17,8 +17,6 @@ package kafka.api -import java.lang.{Integer, IllegalArgumentException} - import org.apache.kafka.clients.producer._ import org.scalatest.junit.JUnit3Suite import org.junit.Test @@ -56,8 +54,10 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { super.setUp() // TODO: we need to migrate to new consumers when 0.9 is final - consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "") - consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "") + val endpoint1 = configs(0).listeners.values.head + val endpoint2 = configs(1).listeners.values.head + consumer1 = new SimpleConsumer("localhost", endpoint1.port, 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", endpoint2.port, 100, 1024*1024, "") } override def tearDown() { diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index a106379..13a865c 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -1,5 +1,6 @@ package other.kafka +import kafka.cluster.SecurityProtocol import org.I0Itec.zkclient.ZkClient import kafka.api._ import kafka.utils.{ShutdownableThread, ZKStringSerializer} @@ -110,7 +111,7 @@ object TestOffsetManager { private val fetchTimer = new KafkaTimer(timer) private val channels = mutable.Map[Int, BlockingChannel]() - private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SecurityProtocol.PLAINTEXT, SocketTimeoutMs) private val numErrors = new AtomicInteger(0) @@ -156,7 +157,7 @@ object TestOffsetManager { println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port)) metadataChannel.disconnect() println("Creating new query channel.") - metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs) + metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SecurityProtocol.PLAINTEXT, SocketTimeoutMs) } finally { Thread.sleep(fetchIntervalMs) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 4d36b8b..bc4aef3 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -65,14 +65,14 @@ class KafkaTest { assertEquals(2, config2.brokerId) // We should be also able to set completely new property - val config3 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987")) + val config3 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact")) assertEquals(1, config3.brokerId) - assertEquals(1987, config3.port) + assertEquals("compact", config3.logCleanupPolicy) // We should be also able to set several properties - val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987", "--override", "broker.id=2")) + val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "log.cleanup.policy=compact", "--override", "broker.id=2")) assertEquals(2, config4.brokerId) - assertEquals(1987, config4.port) + assertEquals("compact", config4.logCleanupPolicy) } @Test(expected = classOf[ExitCalled]) diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 8bc1785..d50421b 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -22,7 +22,7 @@ import kafka.zk.ZooKeeperTestHarness import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} -import kafka.cluster.Broker +import kafka.cluster.{SecurityProtocol, Broker} import kafka.client.ClientUtils import kafka.server.{KafkaConfig, KafkaServer} @@ -61,7 +61,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { val server4 = TestUtils.createServer(KafkaConfig.fromProps(configProps4)) servers ++= List(server1, server2, server3, server4) - brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.config.port)) + brokers = servers.map(s => new Broker(s.config.brokerId, s.config.listeners)) // create topics first createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) @@ -109,7 +109,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic1, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers, "AddPartitionsTest-testIncrementPartitions", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic1), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testIncrementPartitions", 2000,0).topicsMetadata val metaDataForTopic1 = metadata.filter(p => p.topic.equals(topic1)) val partitionDataForTopic1 = metaDataForTopic1.head.partitionsMetadata @@ -134,7 +134,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { // read metadata from a broker and verify the new topic partitions exist TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 1) TestUtils.waitUntilMetadataIsPropagated(servers, topic2, 2) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers, "AddPartitionsTest-testManualAssignmentOfReplicas", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic2), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testManualAssignmentOfReplicas", 2000,0).topicsMetadata val metaDataForTopic2 = metadata.filter(p => p.topic.equals(topic2)) val partitionDataForTopic2 = metaDataForTopic2.head.partitionsMetadata @@ -158,7 +158,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 5) TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 6) - val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers, "AddPartitionsTest-testReplicaPlacement", + val metadata = ClientUtils.fetchTopicMetadata(Set(topic3), brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), "AddPartitionsTest-testReplicaPlacement", 2000,0).topicsMetadata val metaDataForTopic3 = metadata.filter(p => p.topic.equals(topic3)).head diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index fba852a..978eb1e 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -22,12 +22,10 @@ import org.scalatest.junit.JUnitSuite import junit.framework.Assert._ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, SecurityProtocol, EndPoint, Broker} import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} import kafka.utils.SystemTime import org.apache.kafka.common.requests._ -import org.apache.kafka.common.protocol.ApiKeys -import scala.Some import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition import org.apache.kafka.common.TopicPartition @@ -80,21 +78,47 @@ object SerializationTestUtils { TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) ) - private val brokers = List(new Broker(0, "localhost", 1011), new Broker(1, "localhost", 1012), new Broker(2, "localhost", 1013)) - private val partitionMetaData0 = new PartitionMetadata(0, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 0) - private val partitionMetaData1 = new PartitionMetadata(1, Some(brokers.head), replicas = brokers, isr = brokers.tail, errorCode = 1) - private val partitionMetaData2 = new PartitionMetadata(2, Some(brokers.head), replicas = brokers, isr = brokers, errorCode = 2) - private val partitionMetaData3 = new PartitionMetadata(3, Some(brokers.head), replicas = brokers, isr = brokers.tail.tail, errorCode = 3) + private val brokers = List(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1011, SecurityProtocol.PLAINTEXT))), + new Broker(1, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1012, SecurityProtocol.PLAINTEXT))), + new Broker(2, Map(SecurityProtocol.PLAINTEXT -> EndPoint("localhost", 1013, SecurityProtocol.PLAINTEXT)))) + private val brokerEndpoints = brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) + + private val partitionMetaData0 = new PartitionMetadata(0, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 0) + private val partitionMetaData1 = new PartitionMetadata(1, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail, errorCode = 1) + private val partitionMetaData2 = new PartitionMetadata(2, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints, errorCode = 2) + private val partitionMetaData3 = new PartitionMetadata(3, Some(brokerEndpoints.head), replicas = brokerEndpoints, isr = brokerEndpoints.tail.tail, errorCode = 3) private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3) private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq) private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq) + private val leaderAndIsr0 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.map(_.id)) + private val leaderAndIsr1 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.tail.map(_.id)) + private val leaderAndIsr2 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.map(_.id)) + private val leaderAndIsr3 = new LeaderAndIsr(leader = brokers.head.id, isr = brokers.tail.map(_.id)) + + private val leaderIsrAndControllerEpoch0 = new LeaderIsrAndControllerEpoch(leaderAndIsr0, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch1 = new LeaderIsrAndControllerEpoch(leaderAndIsr1, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch2 = new LeaderIsrAndControllerEpoch(leaderAndIsr2, controllerEpoch = 0) + private val leaderIsrAndControllerEpoch3 = new LeaderIsrAndControllerEpoch(leaderAndIsr3, controllerEpoch = 0) + + private val partitionStateInfo0 = new PartitionStateInfo(leaderIsrAndControllerEpoch0, brokers.map(_.id).toSet) + private val partitionStateInfo1 = new PartitionStateInfo(leaderIsrAndControllerEpoch1, brokers.map(_.id).toSet) + private val partitionStateInfo2 = new PartitionStateInfo(leaderIsrAndControllerEpoch2, brokers.map(_.id).toSet) + private val partitionStateInfo3 = new PartitionStateInfo(leaderIsrAndControllerEpoch3, brokers.map(_.id).toSet) + + private val updateMetadataRequestPartitionStateInfo = collection.immutable.Map( + TopicAndPartition(topic1,0) -> partitionStateInfo0, + TopicAndPartition(topic1,1) -> partitionStateInfo1, + TopicAndPartition(topic1,2) -> partitionStateInfo2, + TopicAndPartition(topic1,3) -> partitionStateInfo3 + ) + def createTestLeaderAndIsrRequest() : LeaderAndIsrRequest = { val leaderAndIsr1 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader1, 1, isr1, 1), 1) val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) val map = Map(((topic1, 0), PartitionStateInfo(leaderAndIsr1, isr1.toSet)), ((topic2, 0), PartitionStateInfo(leaderAndIsr2, isr2.toSet))) - new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[Broker](), 0, 1, 0, "") + new LeaderAndIsrRequest(map.toMap, collection.immutable.Set[BrokerEndpoint](), 0, 1, 0, "") } def createTestLeaderAndIsrResponse() : LeaderAndIsrResponse = { @@ -148,7 +172,7 @@ object SerializationTestUtils { } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)).toVector, Seq(topicmetaData1, topicmetaData2), 1) } def createTestOffsetCommitRequestV1: OffsetCommitRequest = { @@ -192,7 +216,7 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError, 0) + ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0) } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { @@ -216,6 +240,23 @@ object SerializationTestUtils { val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1))) JoinGroupResponseAndHeader(1, body) } + + def createUpdateMetadataRequest(versionId: Short): UpdateMetadataRequest = { + UpdateMetadataRequest( + versionId, + correlationId = 0, + clientId = "client1", + controllerId = 0, + controllerEpoch = 0, + partitionStateInfos = updateMetadataRequestPartitionStateInfo, + brokers.toSet + ) + } + + def createUpdateMetadataResponse: UpdateMetadataResponse = { + UpdateMetadataResponse( correlationId = 0, errorCode = 0) + } + } class RequestResponseSerializationTest extends JUnitSuite { @@ -242,6 +283,9 @@ class RequestResponseSerializationTest extends JUnitSuite { private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader private val joinGroupResponse = SerializationTestUtils.createJoinGroupResponseAndHeader + private val updateMetadataRequestV0 = SerializationTestUtils.createUpdateMetadataRequest(0) + private val updateMetadataRequestV1 = SerializationTestUtils.createUpdateMetadataRequest(1) + private val updateMetdataResponse = SerializationTestUtils.createUpdateMetadataResponse @Test def testSerializationAndDeserialization() { @@ -254,7 +298,8 @@ class RequestResponseSerializationTest extends JUnitSuite { offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, consumerMetadataRequest, consumerMetadataResponse, consumerMetadataResponseNoCoordinator, heartbeatRequest, - heartbeatResponse, joinGroupRequest, joinGroupResponse) + heartbeatResponse, joinGroupRequest, joinGroupResponse, + updateMetadataRequestV0, updateMetadataRequestV1, updateMetdataResponse) requestsAndResponses.foreach { original => val buffer = ByteBuffer.allocate(original.sizeInBytes) diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index 995397b..5b7f029 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -50,7 +50,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val group = "group1" val consumer0 = "consumer0" val consumedOffset = 5 - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId,c.listeners))) val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, 0, diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 3093e45..8f38747 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -28,7 +28,6 @@ import org.scalatest.junit.JUnit3Suite import kafka.consumer._ import kafka.serializer._ import kafka.producer.{KeyedMessage, Producer} -import kafka.utils.TestUtils._ import kafka.utils.TestUtils class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { @@ -39,7 +38,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { yield KafkaConfig.fromProps(props) val messages = new mutable.HashMap[Int, Seq[Array[Byte]]] val topic = "topic" - val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, "localhost", c.port))) + val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.listeners))) val shutdown = ZookeeperConsumerConnector.shutdownCommand val queue = new LinkedBlockingQueue[FetchedDataChunk] val topicInfos = configs.map(c => new PartitionTopicInfo(topic, @@ -54,7 +53,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { override def setUp() { super.setUp - createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) + TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers) fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient) fetcher.stopConnections() diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 062790f..d5e4bc3 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -18,11 +18,13 @@ package kafka.integration import java.util.Arrays + import scala.collection.mutable.Buffer import kafka.server._ -import kafka.utils.{Utils, TestUtils} +import kafka.utils.Utils import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness +import kafka.cluster.SecurityProtocol import kafka.common.KafkaException import kafka.utils.TestUtils @@ -38,7 +40,10 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) - def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",") + def bootstrapUrl = configs.map(c => { + val endpoint = c.listeners(SecurityProtocol.PLAINTEXT) + endpoint.host + ":" + endpoint.port + }).mkString(",") override def setUp() { super.setUp diff --git a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala index a671af4..c0c9e76 100644 --- a/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala @@ -22,8 +22,8 @@ import kafka.zk.ZooKeeperTestHarness import kafka.admin.AdminUtils import java.nio.ByteBuffer import junit.framework.Assert._ -import kafka.cluster.Broker -import kafka.utils.TestUtils +import kafka.cluster.{SecurityProtocol, Broker} +import kafka.utils.{Utils, TestUtils} import kafka.utils.TestUtils._ import kafka.server.{KafkaServer, KafkaConfig} import kafka.api.TopicMetadataRequest @@ -34,7 +34,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val props = createBrokerConfigs(1) val configs = props.map(p => KafkaConfig.fromProps(p)) private var server1: KafkaServer = null - val brokers = configs.map(c => new Broker(c.brokerId,c.hostName,c.port)) + val brokerEndPoints = configs.map(c => new Broker(c.brokerId,c.listeners).getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) override def setUp() { super.setUp() @@ -67,7 +67,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = "test" createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) @@ -87,7 +87,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { createTopic(zkClient, topic2, numPartitions = 1, replicationFactor = 1, servers = Seq(server1)) // issue metadata request with empty list of topics - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokers, "TopicMetadataTest-testGetAllTopicMetadata", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokerEndPoints, "TopicMetadataTest-testGetAllTopicMetadata", 2000, 0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(2, topicsMetadata.size) @@ -106,7 +106,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { def testAutoCreateTopic { // auto create topic val topic = "testAutoCreateTopic" - var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testAutoCreateTopic", + var topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testAutoCreateTopic", 2000,0).topicsMetadata assertEquals(ErrorMapping.LeaderNotAvailableCode, topicsMetadata.head.errorCode) assertEquals("Expecting metadata only for 1 topic", 1, topicsMetadata.size) @@ -118,7 +118,7 @@ class TopicMetadataTest extends JUnit3Suite with ZooKeeperTestHarness { TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0) // retry the metadata for the auto created topic - topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic),brokers,"TopicMetadataTest-testBasicTopicMetadata", + topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata", 2000,0).topicsMetadata assertEquals(ErrorMapping.NoError, topicsMetadata.head.errorCode) assertEquals(ErrorMapping.NoError, topicsMetadata.head.partitionsMetadata.head.errorCode) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 8cd5f2f..5cbb61b 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -32,12 +32,12 @@ class LogTest extends JUnitSuite { var logDir: File = null val time = new MockTime(0) var config: KafkaConfig = null - val logConfig = LogConfig() + val logConfig = LogConfig() @Before def setUp() { logDir = TestUtils.tempDir() - val props = TestUtils.createBrokerConfig(0, -1) + val props = TestUtils.createBrokerConfig(0, 1) config = KafkaConfig.fromProps(props) } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 0af23ab..4271601 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,6 +19,8 @@ package kafka.network; import java.net._ import java.io._ +import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.cluster.{SecurityProtocol, EndPoint} import org.junit._ import org.scalatest.junit.JUnitSuite import java.util.Random @@ -31,12 +33,17 @@ import kafka.message.ByteBufferMessageSet import java.nio.channels.SelectionKey import kafka.utils.TestUtils import scala.collection.Map +import scala.collection.JavaConversions._ class SocketServerTest extends JUnitSuite { + val ports = kafka.utils.TestUtils.choosePorts(2) + val plaintextPort = ports.head + val tracePort = ports.last + val server: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, plaintextPort, SecurityProtocol.PLAINTEXT), + SecurityProtocol.TRACE -> EndPoint(null, tracePort, SecurityProtocol.TRACE)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, @@ -73,7 +80,10 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s:SocketServer = server) = new Socket("localhost", s.port) + + def connect(s:SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + new Socket("localhost", server.endpoints.get(protocol).get.port) + } @After def cleanup() { @@ -81,7 +91,8 @@ class SocketServerTest extends JUnitSuite { } @Test def simpleRequest() { - val socket = connect() + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) val correlationId = -1 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -95,9 +106,15 @@ class SocketServerTest extends JUnitSuite { val serializedBytes = new Array[Byte](byteBuffer.remaining) byteBuffer.get(serializedBytes) - sendRequest(socket, 0, serializedBytes) + // Test PLAINTEXT socket + sendRequest(plainSocket, 0, serializedBytes) + processRequest(server.requestChannel) + assertEquals(serializedBytes.toSeq, receiveResponse(plainSocket).toSeq) + + // Test TRACE socket + sendRequest(traceSocket, 0, serializedBytes) processRequest(server.requestChannel) - assertEquals(serializedBytes.toSeq, receiveResponse(socket).toSeq) + assertEquals(serializedBytes.toSeq, receiveResponse(traceSocket).toSeq) } @Test(expected = classOf[IOException]) @@ -129,21 +146,38 @@ class SocketServerTest extends JUnitSuite { "Socket key should be available for reads") } - @Test(expected = classOf[IOException]) + @Test def testSocketsCloseOnShutdown() { // open a connection - val socket = connect() + val plainSocket = connect(protocol = SecurityProtocol.PLAINTEXT) + val traceSocket = connect(protocol = SecurityProtocol.TRACE) val bytes = new Array[Byte](40) // send a request first to make sure the connection has been picked up by the socket server - sendRequest(socket, 0, bytes) + sendRequest(plainSocket, 0, bytes) + sendRequest(traceSocket, 0, bytes) processRequest(server.requestChannel) + + // make sure the sockets are open + server.acceptors.values().map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) // then shutdown the server server.shutdown() val largeChunkOfBytes = new Array[Byte](1000000) // doing a subsequent send should throw an exception as the connection should be closed. // send a large chunk of bytes to trigger a socket flush - sendRequest(socket, 0, largeChunkOfBytes) + try { + sendRequest(plainSocket, 0, largeChunkOfBytes) + fail("expected exception when writing to closed plain socket") + } catch { + case e: IOException => // expected + } + + try { + sendRequest(traceSocket, 0, largeChunkOfBytes) + fail("expected exception when writing to closed trace socket") + } catch { + case e: IOException => // expected + } } @Test @@ -161,8 +195,7 @@ class SocketServerTest extends JUnitSuite { val overrideNum = 6 val overrides: Map[String, Int] = Map("localhost" -> overrideNum) val overrideServer: SocketServer = new SocketServer(0, - host = null, - port = kafka.utils.TestUtils.choosePort, + Map(SecurityProtocol.PLAINTEXT -> EndPoint(null, kafka.utils.TestUtils.choosePort, SecurityProtocol.PLAINTEXT)), numProcessorThreads = 1, maxQueuedRequests = 50, sendBufferSize = 300000, diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index be90c5b..60a961d 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -23,7 +23,7 @@ import junit.framework.Assert._ import org.easymock.EasyMock import org.junit.Test import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{BrokerEndpoint, Broker} import kafka.common._ import kafka.message._ import kafka.producer.async._ @@ -163,8 +163,8 @@ class AsyncProducerTest extends JUnit3Suite { val props = new Properties() props.put("metadata.broker.list", TestUtils.getBrokerListStrFromConfigs(configs)) - val broker1 = new Broker(0, "localhost", 9092) - val broker2 = new Broker(1, "localhost", 9093) + val broker1 = new BrokerEndpoint(0, "localhost", 9092) + val broker2 = new BrokerEndpoint(1, "localhost", 9093) // form expected partitions metadata val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2)) @@ -467,7 +467,7 @@ class AsyncProducerTest extends JUnit3Suite { } private def getTopicMetadata(topic: String, partition: Seq[Int], brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = { - val broker1 = new Broker(brokerId, brokerHost, brokerPort) + val broker1 = new BrokerEndpoint(brokerId, brokerHost, brokerPort) new TopicMetadata(topic, partition.map(new PartitionMetadata(_, Some(broker1), List(broker1)))) } diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index b5208a5..256c274 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,6 +21,7 @@ import java.net.SocketTimeoutException import java.util.Properties import junit.framework.Assert import kafka.admin.AdminUtils +import kafka.cluster.SecurityProtocol import kafka.integration.KafkaServerTestHarness import kafka.message._ import kafka.server.KafkaConfig @@ -39,7 +40,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testReachableServer() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val firstStart = SystemTime.milliseconds @@ -74,7 +76,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testEmptyProduceRequest() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port + val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId @@ -91,7 +94,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers) @@ -118,8 +122,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) props.put("request.required.acks", "0") val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -145,7 +150,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -191,7 +197,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) @@ -217,7 +224,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceRequestWithNoResponse() { val server = servers.head - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) val correlationId = 0 val clientId = SyncProducerConfig.DefaultClientId val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs @@ -232,8 +240,9 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { def testNotEnoughReplicas() { val topicName = "minisrtest" val server = servers.head + val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val props = TestUtils.getSyncProducerConfig(port) - val props = TestUtils.getSyncProducerConfig(server.socketServer.port) props.put("request.required.acks", "-1") val producer = new SyncProducer(new SyncProducerConfig(props)) diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index 296e2b5..9f48104 100644 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -17,6 +17,7 @@ package kafka.server +import kafka.cluster.SecurityProtocol import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ @@ -31,9 +32,8 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) - + props.put("advertised.listeners", SecurityProtocol.PLAINTEXT.toString+"://"+advertisedHostName+":"+advertisedPort.toString) + server = TestUtils.createServer(KafkaConfig.fromProps(props)) } @@ -45,8 +45,9 @@ class AdvertiseBrokerTest extends JUnit3Suite with ZooKeeperTestHarness { def testBrokerAdvertiseToZK { val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId) - assertEquals(advertisedHostName, brokerInfo.get.host) - assertEquals(advertisedPort, brokerInfo.get.port) + val endpoint = brokerInfo.get.endPoints.get(SecurityProtocol.PLAINTEXT).get + assertEquals(advertisedHostName, endpoint.host) + assertEquals(advertisedPort, endpoint.port) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 191251d..ee86923 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -18,9 +18,11 @@ package unit.kafka.server import java.util.Properties +import kafka.cluster.SecurityProtocol import kafka.message._ import kafka.server.{Defaults, KafkaConfig} import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.protocol.ApiVersion import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite @@ -173,8 +175,10 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.PortProp => expected.setProperty(name, "1234") case KafkaConfig.HostNameProp => expected.setProperty(name, nextString(10)) + case KafkaConfig.ListenersProp => expected.setProperty(name, "PLAINTEXT://:9092") case KafkaConfig.AdvertisedHostNameProp => expected.setProperty(name, nextString(10)) case KafkaConfig.AdvertisedPortProp => expected.setProperty(name, "4321") + case KafkaConfig.AdvertisedListenersProp => expected.setProperty(name, "PLAINTEXT://:2909") case KafkaConfig.SocketRequestMaxBytesProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.MaxConnectionsPerIpProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.MaxConnectionsPerIpOverridesProp => expected.setProperty(name, "127.0.0.1:2, 127.0.0.2:3") @@ -205,6 +209,9 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.MinInSyncReplicasProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case KafkaConfig.IntraBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) + case KafkaConfig.IntraBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.getLatestVersion.toString) + case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.OffsetsTopicPartitionsProp => expected.setProperty(name, atLeastOneIntProp) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 7f47e6f..0d61057 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -17,13 +17,15 @@ package kafka.server +import java.util.Properties + +import kafka.cluster.SecurityProtocol import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.protocol.ApiVersion import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite -import kafka.utils.TestUtils -import kafka.message.GZIPCompressionCodec -import kafka.message.NoCompressionCodec +import kafka.utils.{Utils, TestUtils} class KafkaConfigTest extends JUnit3Suite { @@ -34,7 +36,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(60L * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -44,7 +45,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -54,7 +54,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -63,7 +62,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -74,7 +72,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -85,7 +82,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis) - } @Test @@ -94,12 +90,13 @@ class KafkaConfigTest extends JUnit3Suite { val hostName = "fake-host" val props = TestUtils.createBrokerConfig(0, port) - props.put("host.name", hostName) - + props.put("listeners", "PLAINTEXT://"+hostName+":"+port) + val serverConfig = KafkaConfig.fromProps(props) - - assertEquals(serverConfig.advertisedHostName, hostName) - assertEquals(serverConfig.advertisedPort, port) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get + assertEquals(endpoint.host, hostName) + assertEquals(endpoint.port, port) } @Test @@ -109,13 +106,99 @@ class KafkaConfigTest extends JUnit3Suite { val advertisedPort = 1234 val props = TestUtils.createBrokerConfig(0, port) - props.put("advertised.host.name", advertisedHostName) - props.put("advertised.port", advertisedPort.toString) - + props.put("advertised.listeners", "PLAINTEXT://"+advertisedHostName+":"+advertisedPort.toString) + val serverConfig = KafkaConfig.fromProps(props) + val endpoints = serverConfig.advertisedListeners + val endpoint = endpoints.get(SecurityProtocol.PLAINTEXT).get - assertEquals(serverConfig.advertisedHostName, advertisedHostName) - assertEquals(serverConfig.advertisedPort, advertisedPort) + assertEquals(endpoint.host, advertisedHostName) + assertEquals(endpoint.port, advertisedPort) + } + + + @Test + def testDuplicateListeners() { + val props = new Properties() + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") + + // listeners with duplicate port + props.put("listeners", "PLAINTEXT://localhost:9091,TRACE://localhost:9091") + assert(!isValidKafkaConfig(props)) + + // listeners with duplicate protocol + props.put("listeners", "PLAINTEXT://localhost:9091,PLAINTEXT://localhost:9092") + assert(!isValidKafkaConfig(props)) + + // advertised listeners with duplicate port + props.put("advertised,listeners", "PLAINTEXT://localhost:9091,TRACE://localhost:9091") + assert(!isValidKafkaConfig(props)) + } + + @Test + def testBadListenerProtocol() { + val props = new Properties() + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") + props.put("listeners", "BAD://localhost:9091") + + assert(!isValidKafkaConfig(props)) + } + + @Test + def testListenerDefaults() { + val props = new Properties() + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") + + // configuration with host and port, but no listeners + props.put("host.name", "myhost") + props.put("port", "1111") + + val conf = KafkaConfig.fromProps(props) + assertEquals(Utils.listenerListToEndPoints("PLAINTEXT://myhost:1111"), conf.listeners) + + // configuration with null host + props.remove("host.name") + + val conf2 = KafkaConfig.fromProps(props) + assertEquals(Utils.listenerListToEndPoints("PLAINTEXT://:1111"), conf2.listeners) + assertEquals(null, conf2.listeners(SecurityProtocol.PLAINTEXT).host) + + // configuration with advertised host and port, and no advertised listeners + props.put("advertised.host.name", "otherhost") + props.put("advertised.port", "2222") + + val conf3 = KafkaConfig.fromProps(props) + assertEquals(conf3.advertisedListeners, Utils.listenerListToEndPoints("PLAINTEXT://otherhost:2222")) + } + + @Test + def testVersionConfiguration() { + val props = new Properties() + props.put("broker.id", "1") + props.put("zookeeper.connect", "localhost:2181") + val conf = KafkaConfig.fromProps(props) + assertEquals(conf.intraBrokerProtocolVersion, ApiVersion.getLatestVersion) + + props.put("use.intra.broker.protocol.version","0.8.2.0") + val conf2 = KafkaConfig.fromProps(props) + assertEquals(conf2.intraBrokerProtocolVersion, ApiVersion.KAFKA_082) + + // check that 0.8.2.0 is the same as 0.8.2.1 + props.put("use.intra.broker.protocol.version","0.8.2.1") + val conf3 = KafkaConfig.fromProps(props) + assertEquals(conf3.intraBrokerProtocolVersion, ApiVersion.KAFKA_082) + } + + private def isValidKafkaConfig(props: Properties): Boolean = { + try { + KafkaConfig.fromProps(props) + true + } catch { + case e: IllegalArgumentException => false + } } @Test @@ -161,7 +244,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis) - } @Test @@ -172,7 +254,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis) - } @Test @@ -181,7 +262,6 @@ class KafkaConfigTest extends JUnit3Suite { val cfg = KafkaConfig.fromProps(props) assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis ) - } @Test diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index f252805..f2dbc6f 100644 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -23,7 +23,7 @@ import kafka.utils.TestUtils._ import junit.framework.Assert._ import kafka.utils.{ZkUtils, Utils, TestUtils} import kafka.controller.{ControllerContext, LeaderIsrAndControllerEpoch, ControllerChannelManager} -import kafka.cluster.Broker +import kafka.cluster.{SecurityProtocol, Broker} import kafka.common.ErrorMapping import kafka.api._ @@ -118,7 +118,8 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { // start another controller val controllerId = 2 val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, TestUtils.choosePort())) - val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.config.port)) + val brokers = servers.map(s => new Broker(s.config.brokerId, s.config.listeners)) + val brokerEndPoints = brokers.map(b => b.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)) val controllerContext = new ControllerContext(zkClient, 6000) controllerContext.liveBrokers = brokers.toSet val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig) @@ -128,7 +129,7 @@ class LeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { leaderAndIsr.put((topic, partitionId), new LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId2, List(brokerId1, brokerId2)), 2)) val partitionStateInfo = leaderAndIsr.mapValues(l => new PartitionStateInfo(l, Set(0,1))).toMap - val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokers.toSet, controllerId, + val leaderAndIsrRequest = new LeaderAndIsrRequest(partitionStateInfo, brokerEndPoints.toSet, controllerId, staleControllerEpoch, 0, "") controllerChannelManager.sendRequest(brokerId2, leaderAndIsrRequest, staleControllerEpochCallback) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 8c9f9e7..d1eee83 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -197,7 +197,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { private def createBrokerConfig(nodeId: Int, port: Int): Properties = { val props = new Properties props.put("broker.id", nodeId.toString) - props.put("port", port.toString) + props.put("listeners", "PLAINTEXT://localhost:" + port.toString) props.put("log.dir", getLogDir.getAbsolutePath) props.put("log.flush.interval.messages", "1") props.put("enable.zookeeper", "false") diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1682a77..69eacab 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -34,7 +34,7 @@ import kafka.server._ import kafka.producer._ import kafka.message._ import kafka.api._ -import kafka.cluster.Broker +import kafka.cluster.{SecurityProtocol, Broker} import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition @@ -148,7 +148,10 @@ object TestUtils extends Logging { } def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { - configs.map(c => formatAddress(c.hostName, c.port)).mkString(",") + configs.map(c => { + val endpoint = c.listeners.get(SecurityProtocol.PLAINTEXT).get + formatAddress(endpoint.host, endpoint.port) + }).mkString(",") } /** @@ -159,8 +162,7 @@ object TestUtils extends Logging { enableDeleteTopic: Boolean = false): Properties = { val props = new Properties if (nodeId >= 0) props.put("broker.id", nodeId.toString) - props.put("host.name", "localhost") - props.put("port", port.toString) + props.put("listeners", "PLAINTEXT://localhost:"+port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") @@ -474,13 +476,13 @@ object TestUtils extends Logging { } def createBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667)) - brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, b.host, b.port, 6000, jmxPort = -1)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) + brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, b.id, "localhost", 6667, b.endPoints, 6000, jmxPort = -1)) brokers } def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667)) + val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b)) brokers } diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index 250ffe0..7a32e8d 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:0", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json index 3f9e7d0..2929414 100644 --- a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json +++ b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json index b10c626..d9818e1 100644 --- a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json +++ b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json index 22c0802..fe42626 100644 --- a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json +++ b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json index 1317d84..37d180a 100644 --- a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json +++ b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json index d313465..dae8f76 100644 --- a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json +++ b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json index bd00f13..b6f513f 100644 --- a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json +++ b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json index ac5c13b..4954752 100644 --- a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json +++ b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json index 6dadd7a..0476b12 100644 --- a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json +++ b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 20480" + "07":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json index 614cb1c..b9517b4 100644 --- a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json index fbf0177..3eb39a2 100644 --- a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json index 317a6e3..6bfc757 100644 --- a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json index d1a790b..1cfe71c 100644 --- a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json index 8f4f8bf..13f3ac0 100644 --- a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json index a307f85..ccd4774 100644 --- a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json index 827319e..b1da75a 100644 --- a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json index ae015cd..359abe7 100644 --- a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json index 1951a12..90ea441 100644 --- a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 20480" + "10":"Log segment size : 10000000" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json index 0c6d7a3..680213f 100644 --- a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json +++ b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "20480", + "log.segment.bytes": "10000000", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index 41d511c..a9b73f7 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -436,6 +436,7 @@ def generate_overriden_props_files(testsuitePathname, testcaseEnv, systemTestEnv addedCSVConfig["kafka.metrics.polling.interval.secs"] = "5" addedCSVConfig["kafka.metrics.reporters"] = "kafka.metrics.KafkaCSVMetricsReporter" addedCSVConfig["kafka.csv.metrics.reporter.enabled"] = "true" + addedCSVConfig["listeners"] = "PLAINTEXT://localhost:"+tcCfg["port"] if brokerVersion == "0.7": addedCSVConfig["brokerid"] = tcCfg["brokerid"] -- 1.9.5 (Apple Git-50.3) From 3d8269b340106587128e513d79ab332b84f8e16c Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 2 Apr 2015 14:49:59 -0700 Subject: [PATCH 3/3] moved ApiVersion to core, fixed SocketServer concurrency issue and other minor things --- .../apache/kafka/common/protocol/ApiVersion.java | 65 ----------- core/src/main/scala/kafka/api/ApiVersion.scala | 66 +++++++++++ core/src/main/scala/kafka/cluster/EndPoint.scala | 11 +- .../controller/ControllerChannelManager.scala | 5 +- .../main/scala/kafka/network/SocketServer.scala | 55 ++++----- core/src/main/scala/kafka/server/KafkaConfig.scala | 19 ++-- .../main/scala/kafka/server/MetadataCache.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 2 +- .../test/scala/unit/kafka/cluster/BrokerTest.scala | 6 +- .../unit/kafka/network/SocketServerTest.scala | 3 +- .../unit/kafka/producer/SyncProducerTest.scala | 8 +- .../kafka/server/KafkaConfigConfigDefTest.scala | 5 +- .../scala/unit/kafka/server/KafkaConfigTest.scala | 16 ++- system_test/README.txt | 5 + system_test/testcase_to_run.json | 122 +++++++++++++++++++- system_test/testcase_to_run_all_replica.json | 123 +++++++++++++++++++++ 16 files changed, 383 insertions(+), 130 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java create mode 100644 core/src/main/scala/kafka/api/ApiVersion.scala create mode 100644 system_test/testcase_to_run_all_replica.json diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java deleted file mode 100644 index fdd8585..0000000 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * 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.kafka.common.protocol; - -/** - * This class contains the different Kafka versions. - * Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves. - * This is only for intra-broker communications - when communicating with clients, the client decides on the API version. - * - * Note that ORDER MATTERS in the enum. - * We consider version A as newer than B if it appears later in the list of constants here. - * If you add new versions, add them in the correct chronological release order. - */ -public enum ApiVersion { - KAFKA_082("0.8.2.X"), - KAFKA_083("0.8.3.X"); - - private final String version; - - public boolean onOrAfter(ApiVersion other) { - return compareTo(other) >= 0; - } - - private ApiVersion(final String version) { - this.version = version; - } - - /* Parse user readable version number. This assumes the convention of 0.8.2.0, 0.8.2.1, 0.8.3.0, 0.9.0.0, etc. - * We are assuming that we will never change API in bug-fix versions (i.e. 0.8.2.1 will have same API as 0.8.2.0) - * */ - public static ApiVersion parseConfig(String version) { - String[] vals = version.split("\\."); - StringBuilder parsed = new StringBuilder(); - parsed.append("KAFKA_"); - - // We only care about the first 3 version digits. Others don't impact the API - parsed.append(vals[0]).append(vals[1]).append(vals[2]); - return ApiVersion.valueOf(parsed.toString()); - } - - public static ApiVersion getLatestVersion() { - ApiVersion[] values = ApiVersion.values(); - return values[values.length - 1]; - } - - @Override - public String toString() { - return version; - } - -} diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala new file mode 100644 index 0000000..d0267de --- /dev/null +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -0,0 +1,66 @@ +/* + * 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 kafka.api + +/** + * This class contains the different Kafka versions. + * Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves. + * This is only for intra-broker communications - when communicating with clients, the client decides on the API version. + * + * Note that the ID we initialize for each version is important. + * We consider a version newer than another, if it has a higher ID (to avoid depending on lexicographic order) + */ +object ApiVersion { + // This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541 + implicit def orderingByVersion[A <: ApiVersion]: Ordering[A] = Ordering.by(_.id) + + private val versionNameMap = Map( + "0.8.2" -> KAFKA_082, + "0.8.3" -> KAFKA_083 + ) + + def apply(version: String): ApiVersion = versionNameMap(version.split("\\.").slice(0,3).mkString(".")) + + def latestVersion = versionNameMap.values.max +} + +sealed trait ApiVersion extends Ordered[ApiVersion] { + val version: String + val id: Int + + override def compare(that: ApiVersion): Int = { + ApiVersion.orderingByVersion.compare(this, that) + } + + def onOrAfter(that: ApiVersion): Boolean = { + this.compare(that) >= 0 + } + + override def toString(): String = version +} + +// Keep the IDs in order of versions +case object KAFKA_082 extends ApiVersion { + val version: String = "0.8.2.X" + val id: Int = 0 +} + +case object KAFKA_083 extends ApiVersion { + val version: String = "0.8.3.X" + val id: Int = 1 +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 1d853b1..752b47f 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.common.KafkaException import kafka.cluster.SecurityProtocol._ +import org.apache.kafka.common.utils.Utils object EndPoint { @@ -54,9 +55,13 @@ object EndPoint { */ case class EndPoint(host: String, port: Int, protocolType: SecurityProtocol) { - override def toString: String = { - val hostStr = if (host == null || host.contains(":")) "[" + host + "]" else host - protocolType + "://" + hostStr + ":" + port + def connectionString(): String = { + val hostport = + if (host == null) + ":"+port + else + Utils.formatAddress(host, port) + protocolType + "://" + hostport } def writeTo(buffer: ByteBuffer): Unit = { diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 0fc02b7..ddffb11 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -18,15 +18,12 @@ package kafka.controller import kafka.network.{Receive, BlockingChannel} import kafka.utils.{Utils, Logging, ShutdownableThread} -import org.apache.kafka.common.protocol.ApiVersion import collection.mutable.HashMap import kafka.cluster.Broker import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} import kafka.server.KafkaConfig import collection.mutable import kafka.api._ -import org.apache.log4j.Logger -import scala.Some import kafka.common.TopicAndPartition import kafka.api.RequestOrResponse import collection.Set @@ -302,7 +299,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap - val versionId = if (controller.config.intraBrokerProtocolVersion.onOrAfter(ApiVersion.KAFKA_083)) 1 else 0 + val versionId = if (controller.config.intraBrokerProtocolVersion.onOrAfter(KAFKA_083)) 1 else 0 val updateMetadataRequest = new UpdateMetadataRequest(versionId = versionId.toShort, controllerId = controllerId, controllerEpoch = controllerEpoch, correlationId = correlationId, clientId = clientId, partitionStateInfos = partitionStateInfos, aliveBrokers = controllerContext.liveOrShuttingDownBrokers) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 7069da0..ad4d633 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -28,7 +28,6 @@ import kafka.cluster.EndPoint import kafka.cluster.SecurityProtocol.SecurityProtocol import scala.collection._ -import scala.collection.JavaConversions._ import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup @@ -54,7 +53,7 @@ class SocketServer(val brokerId: Int, this.logIdent = "[Socket Server on Broker " + brokerId + "], " private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) - @volatile private[network] var acceptors: ConcurrentHashMap[EndPoint,Acceptor] = new ConcurrentHashMap[EndPoint,Acceptor]() + private[network] var acceptors = mutable.Map[EndPoint,Acceptor]() val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) /* a meter to track the average free capacity of the network processors */ @@ -75,18 +74,20 @@ class SocketServer(val brokerId: Int, */ def startup() { val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) - for(i <- 0 until numProcessorThreads) { - processors(i) = new Processor(i, - time, - maxRequestSize, - aggregateIdleMeter, - newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), - numProcessorThreads, - requestChannel, - quotas, - connectionsMaxIdleMs, - portToProtocol) - Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() + this.synchronized { + for (i <- 0 until numProcessorThreads) { + processors(i) = new Processor(i, + time, + maxRequestSize, + aggregateIdleMeter, + newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), + numProcessorThreads, + requestChannel, + quotas, + connectionsMaxIdleMs, + portToProtocol) + Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() + } } newGauge("ResponsesBeingSent", new Gauge[Int] { @@ -100,14 +101,16 @@ class SocketServer(val brokerId: Int, // right now we will use the same processors for all ports, since we didn't implement different protocols // in the future, we may implement different processors for SSL and Kerberos - endpoints.values.foreach(endpoint => { - val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas) - acceptors.put(endpoint,acceptor) - Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() - acceptor.awaitStartup - }) + this.synchronized { + endpoints.values.foreach(endpoint => { + val acceptor = new Acceptor(endpoint.host, endpoint.port, processors, sendBufferSize, recvBufferSize, quotas) + acceptors.put(endpoint, acceptor) + Utils.newThread("kafka-socket-acceptor-%s-%d".format(endpoint.protocolType.toString, endpoint.port), acceptor, false).start() + acceptor.awaitStartup + }) + } - info("Started " + acceptors.size() + " acceptor threads") + info("Started " + acceptors.size + " acceptor threads") } /** @@ -115,10 +118,12 @@ class SocketServer(val brokerId: Int, */ def shutdown() = { info("Shutting down") - if(acceptors != null) - acceptors.values().foreach(_.shutdown()) - for(processor <- processors) - processor.shutdown() + this.synchronized { + if (acceptors != null) + acceptors.values.foreach(_.shutdown()) + for (processor <- processors) + processor.shutdown() + } info("Shutdown completed") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index b371ff6..dbeb120 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,16 +19,13 @@ package kafka.server import java.util.Properties -import kafka.cluster.SecurityProtocol.SecurityProtocol +import kafka.api.ApiVersion import kafka.cluster.{SecurityProtocol, EndPoint} import kafka.cluster.SecurityProtocol.SecurityProtocol -import kafka.common.InvalidConfigException import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.Utils import org.apache.kafka.common.config.ConfigDef -import org.apache.kafka.common.protocol.ApiVersion - import scala.collection.{immutable, JavaConversions, Map} object Defaults { @@ -108,7 +105,7 @@ object Defaults { val LeaderImbalanceCheckIntervalSeconds = 300 val UncleanLeaderElectionEnable = true val IntraBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString - val IntraBrokerProtocolVersion = ApiVersion.getLatestVersion.toString + val IntraBrokerProtocolVersion = ApiVersion.latestVersion.toString /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetries = 3 @@ -219,7 +216,7 @@ object KafkaConfig { val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds" val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" val IntraBrokerSecurityProtocolProp = "security.intra.broker.protocol" - val IntraBrokerProtocolVersionProp = "use.intra.broker.protocol.version" + val IntraBrokerProtocolVersionProp = "intra.broker.protocol.version" /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" @@ -344,7 +341,7 @@ object KafkaConfig { val IntraBrokerSecurityProtocolDoc = "Security protocol used to communicate between brokers. Defaults to plain text." val IntraBrokerProtocolVersionDoc = "Specify which version of the inter-broker protocol will be used.\n" + " This is typically bumped after all brokers were upgraded to a new version.\n" + - " Example of some valid values are: 0.8.2.0, 0.8.2.1, 0.8.3.0. Check ApiVersion for the full list." + " Example of some valid values are: 0.8.2, 0.8.2.0, 0.8.2.1, 0.8.3, 0.8.3.0. Check ApiVersion for the full list." /** ********* Controlled shutdown configuration ***********/ val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." @@ -590,7 +587,7 @@ object KafkaConfig { leaderImbalanceCheckIntervalSeconds = parsed.get(LeaderImbalanceCheckIntervalSecondsProp).asInstanceOf[Int], uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], intraBrokerSecurityProtocol = SecurityProtocol.withName(parsed.get(IntraBrokerSecurityProtocolProp).asInstanceOf[String]), - intraBrokerProtocolVersion = ApiVersion.parseConfig(parsed.get(IntraBrokerProtocolVersionProp).asInstanceOf[String]), + intraBrokerProtocolVersion = ApiVersion(parsed.get(IntraBrokerProtocolVersionProp).asInstanceOf[String]), /** ********* Controlled shutdown configuration ***********/ controlledShutdownMaxRetries = parsed.get(ControlledShutdownMaxRetriesProp).asInstanceOf[Int], controlledShutdownRetryBackoffMs = parsed.get(ControlledShutdownRetryBackoffMsProp).asInstanceOf[Int], @@ -734,7 +731,7 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val leaderImbalanceCheckIntervalSeconds: Int = Defaults.LeaderImbalanceCheckIntervalSeconds, val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, val intraBrokerSecurityProtocol: SecurityProtocol = SecurityProtocol.withName(Defaults.IntraBrokerSecurityProtocol), - val intraBrokerProtocolVersion: ApiVersion = ApiVersion.parseConfig(Defaults.IntraBrokerProtocolVersion), + val intraBrokerProtocolVersion: ApiVersion = ApiVersion(Defaults.IntraBrokerProtocolVersion), /** ********* Controlled shutdown configuration ***********/ val controlledShutdownMaxRetries: Int = Defaults.ControlledShutdownMaxRetries, @@ -794,7 +791,7 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ } } - def validateUniquePortAndProtocol(listeners: String) { + private def validateUniquePortAndProtocol(listeners: String) { val endpoints = try { val listenerList = Utils.parseCsvList(listeners) @@ -810,7 +807,7 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ } // If the user did not define listeners but did define host or port, let's use them in backward compatible way - // If none of those are defined, we default to PLAINTEXT://null:6667 + // If none of those are defined, we default to PLAINTEXT://:9092 private def getListeners(): immutable.Map[SecurityProtocol, EndPoint] = { if (_listeners.isDefined) { validateUniquePortAndProtocol(_listeners.get) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 675e9c9..2f59314 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -67,7 +67,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { if (!leaderBrokerInfo.isDefined) throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) else - leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) + leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).map(_.getBrokerEndPoint(protocol)) if (replicaInfo.size < replicas.size) throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 6f0294f..0f276c8 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -187,7 +187,7 @@ object ZkUtils extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.toString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.connectionString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) val expectedBroker = new Broker(id, advertisedEndpoints) registerBrokerInZk(zkClient, brokerIdPath, brokerInfo, expectedBroker, timeout) diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala index 3df8aca..a619a0c 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerTest.scala @@ -106,19 +106,19 @@ class BrokerTest extends JUnit3Suite with Logging { var endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == "localhost") assert(endpoint.port == 9092) - assert(endpoint.toString == "PLAINTEXT://localhost:9092") + assert(endpoint.connectionString == "PLAINTEXT://localhost:9092") // also test for default bind connectionString = "PLAINTEXT://:9092" endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == null) assert(endpoint.port == 9092) - assert(endpoint.toString == "PLAINTEXT://[null]:9092") + assert(endpoint.connectionString == "PLAINTEXT://:9092") // also test for ipv6 connectionString = "PLAINTEXT://[::1]:9092" endpoint = EndPoint.createEndPoint(connectionString) assert(endpoint.host == "::1") assert(endpoint.port == 9092) - assert(endpoint.toString == "PLAINTEXT://[::1]:9092") + assert(endpoint.connectionString == "PLAINTEXT://[::1]:9092") } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 4271601..2b34e5d 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -33,7 +33,6 @@ import kafka.message.ByteBufferMessageSet import java.nio.channels.SelectionKey import kafka.utils.TestUtils import scala.collection.Map -import scala.collection.JavaConversions._ class SocketServerTest extends JUnitSuite { @@ -158,7 +157,7 @@ class SocketServerTest extends JUnitSuite { processRequest(server.requestChannel) // make sure the sockets are open - server.acceptors.values().map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) + server.acceptors.values.map(acceptor => assertFalse(acceptor.serverChannel.socket.isClosed)) // then shutdown the server server.shutdown() diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 3eb7110..9b6d694 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -40,7 +40,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testReachableServer() { val server = servers.head - val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -94,7 +94,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testMessageSizeTooLarge() { val server = servers.head - val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -150,7 +150,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testProduceCorrectlyReceivesResponse() { val server = servers.head - val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) @@ -197,7 +197,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val timeoutMs = 500 val server = servers.head - val port = server.socketServer.endpoints.get(SecurityProtocol.PLAINTEXT).get.port + val port = server.socketServer.endpoints(SecurityProtocol.PLAINTEXT).port val props = TestUtils.getSyncProducerConfig(port) val producer = new SyncProducer(new SyncProducerConfig(props)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index ee86923..04d2556 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -18,11 +18,10 @@ package unit.kafka.server import java.util.Properties +import kafka.api.ApiVersion import kafka.cluster.SecurityProtocol import kafka.message._ import kafka.server.{Defaults, KafkaConfig} -import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.protocol.ApiVersion import org.junit.{Assert, Test} import org.scalatest.junit.JUnit3Suite @@ -210,7 +209,7 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.AutoLeaderRebalanceEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.IntraBrokerSecurityProtocolProp => expected.setProperty(name, SecurityProtocol.PLAINTEXT.toString) - case KafkaConfig.IntraBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.getLatestVersion.toString) + case KafkaConfig.IntraBrokerProtocolVersionProp => expected.setProperty(name, ApiVersion.latestVersion.toString) case KafkaConfig.ControlledShutdownEnableProp => expected.setProperty(name, randFrom("true", "false")) case KafkaConfig.OffsetsLoadBufferSizeProp => expected.setProperty(name, atLeastOneIntProp) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 0d61057..2fc04fc 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -19,9 +19,9 @@ package kafka.server import java.util.Properties +import kafka.api.{KAFKA_082, ApiVersion} import kafka.cluster.SecurityProtocol import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.protocol.ApiVersion import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite @@ -164,6 +164,7 @@ class KafkaConfigTest extends JUnit3Suite { val conf2 = KafkaConfig.fromProps(props) assertEquals(Utils.listenerListToEndPoints("PLAINTEXT://:1111"), conf2.listeners) + assertEquals(Utils.listenerListToEndPoints("PLAINTEXT://:1111"), conf2.advertisedListeners) assertEquals(null, conf2.listeners(SecurityProtocol.PLAINTEXT).host) // configuration with advertised host and port, and no advertised listeners @@ -180,16 +181,19 @@ class KafkaConfigTest extends JUnit3Suite { props.put("broker.id", "1") props.put("zookeeper.connect", "localhost:2181") val conf = KafkaConfig.fromProps(props) - assertEquals(conf.intraBrokerProtocolVersion, ApiVersion.getLatestVersion) + assertEquals(ApiVersion.latestVersion, conf.intraBrokerProtocolVersion) - props.put("use.intra.broker.protocol.version","0.8.2.0") + props.put("intra.broker.protocol.version","0.8.2.0") val conf2 = KafkaConfig.fromProps(props) - assertEquals(conf2.intraBrokerProtocolVersion, ApiVersion.KAFKA_082) + assertEquals(KAFKA_082, conf2.intraBrokerProtocolVersion) // check that 0.8.2.0 is the same as 0.8.2.1 - props.put("use.intra.broker.protocol.version","0.8.2.1") + props.put("intra.broker.protocol.version","0.8.2.1") val conf3 = KafkaConfig.fromProps(props) - assertEquals(conf3.intraBrokerProtocolVersion, ApiVersion.KAFKA_082) + assertEquals(KAFKA_082, conf3.intraBrokerProtocolVersion) + + //check that latest is newer than 0.8.2 + assert(ApiVersion.latestVersion.onOrAfter(conf3.intraBrokerProtocolVersion)) } private def isValidKafkaConfig(props: Properties): Boolean = { diff --git a/system_test/README.txt b/system_test/README.txt index 0e469e3..86082b4 100644 --- a/system_test/README.txt +++ b/system_test/README.txt @@ -57,6 +57,11 @@ The framework has the following levels: $ python -u -B system_test_runner.py 2>&1 | tee system_test_output.log 4. To turn on debugging, update system_test/logging.conf by changing the level in handlers session from INFO to DEBUG. + We also have three built-in test suites you can use: + * run_sanity.sh - will run a single basic replication test + * run_all_replica.sh - will run all replication tests + * run_all.sh - will run all replication and mirror_maker tests + # ========================== # Adding Test Case # ========================== diff --git a/system_test/testcase_to_run.json b/system_test/testcase_to_run.json index c6cf17e..34841f5 100644 --- a/system_test/testcase_to_run.json +++ b/system_test/testcase_to_run.json @@ -1,5 +1,123 @@ { - "ReplicaBasicTest" : [ - "testcase_1" + "ReplicaBasicTest" : [ + "testcase_0001", + "testcase_0002", + "testcase_0003", + "testcase_0004", + "testcase_0005", + "testcase_0006", + "testcase_0007", + "testcase_0008", + "testcase_0009", + "testcase_0010", + + "testcase_0021", + "testcase_0022", + "testcase_0023", + + "testcase_0101", + "testcase_0102", + "testcase_0103", + "testcase_0104", + "testcase_0105", + "testcase_0106", + "testcase_0107", + "testcase_0108", + "testcase_0109", + "testcase_0110", + + "testcase_10101", + "testcase_10102", + "testcase_10103", + "testcase_10104", + "testcase_10105", + "testcase_10106", + "testcase_10107", + "testcase_10108", + "testcase_10109", + "testcase_10110", + + "testcase_0111", + "testcase_0112", + "testcase_0113", + "testcase_0114", + "testcase_0115", + "testcase_0116", + "testcase_0117", + "testcase_0118", + + "testcase_0121", + "testcase_0122", + "testcase_0123", + "testcase_0124", + "testcase_0125", + "testcase_0126", + "testcase_0127", + + "testcase_0131", + "testcase_0132", + "testcase_0133", + "testcase_0134", + + "testcase_10131", + "testcase_10132", + "testcase_10133", + "testcase_10134", + + "testcase_0151", + "testcase_0152", + "testcase_0153", + "testcase_0154", + "testcase_0155", + "testcase_0156", + "testcase_0157", + "testcase_0158", + + "testcase_0201", + "testcase_0202", + "testcase_0203", + "testcase_0204", + "testcase_0205", + "testcase_0206", + "testcase_0207", + "testcase_0208", + + "testcase_0251", + "testcase_0252", + "testcase_0253", + "testcase_0254", + "testcase_0255", + "testcase_0256", + "testcase_0257", + "testcase_0258", + + "testcase_0301", + "testcase_0302", + "testcase_0303", + "testcase_0304", + "testcase_0305", + "testcase_0306", + "testcase_0307", + "testcase_0308", + + "testcase_4001", + "testcase_4002", + "testcase_4003", + "testcase_4004", + "testcase_4005", + "testcase_4006", + "testcase_4007", + "testcase_4008", + + "testcase_4011", + "testcase_4012", + "testcase_4013", + "testcase_4014", + "testcase_4015", + "testcase_4016", + "testcase_4017", + "testcase_4018", + + "testcase_9051" ] } diff --git a/system_test/testcase_to_run_all_replica.json b/system_test/testcase_to_run_all_replica.json new file mode 100644 index 0000000..34841f5 --- /dev/null +++ b/system_test/testcase_to_run_all_replica.json @@ -0,0 +1,123 @@ +{ + "ReplicaBasicTest" : [ + "testcase_0001", + "testcase_0002", + "testcase_0003", + "testcase_0004", + "testcase_0005", + "testcase_0006", + "testcase_0007", + "testcase_0008", + "testcase_0009", + "testcase_0010", + + "testcase_0021", + "testcase_0022", + "testcase_0023", + + "testcase_0101", + "testcase_0102", + "testcase_0103", + "testcase_0104", + "testcase_0105", + "testcase_0106", + "testcase_0107", + "testcase_0108", + "testcase_0109", + "testcase_0110", + + "testcase_10101", + "testcase_10102", + "testcase_10103", + "testcase_10104", + "testcase_10105", + "testcase_10106", + "testcase_10107", + "testcase_10108", + "testcase_10109", + "testcase_10110", + + "testcase_0111", + "testcase_0112", + "testcase_0113", + "testcase_0114", + "testcase_0115", + "testcase_0116", + "testcase_0117", + "testcase_0118", + + "testcase_0121", + "testcase_0122", + "testcase_0123", + "testcase_0124", + "testcase_0125", + "testcase_0126", + "testcase_0127", + + "testcase_0131", + "testcase_0132", + "testcase_0133", + "testcase_0134", + + "testcase_10131", + "testcase_10132", + "testcase_10133", + "testcase_10134", + + "testcase_0151", + "testcase_0152", + "testcase_0153", + "testcase_0154", + "testcase_0155", + "testcase_0156", + "testcase_0157", + "testcase_0158", + + "testcase_0201", + "testcase_0202", + "testcase_0203", + "testcase_0204", + "testcase_0205", + "testcase_0206", + "testcase_0207", + "testcase_0208", + + "testcase_0251", + "testcase_0252", + "testcase_0253", + "testcase_0254", + "testcase_0255", + "testcase_0256", + "testcase_0257", + "testcase_0258", + + "testcase_0301", + "testcase_0302", + "testcase_0303", + "testcase_0304", + "testcase_0305", + "testcase_0306", + "testcase_0307", + "testcase_0308", + + "testcase_4001", + "testcase_4002", + "testcase_4003", + "testcase_4004", + "testcase_4005", + "testcase_4006", + "testcase_4007", + "testcase_4008", + + "testcase_4011", + "testcase_4012", + "testcase_4013", + "testcase_4014", + "testcase_4015", + "testcase_4016", + "testcase_4017", + "testcase_4018", + + "testcase_9051" + ] +} -- 1.9.5 (Apple Git-50.3)