[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: alex-topics-to-move.json {topics: [{topic: testingTopic}], version:1 } Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]},
[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]},
[jira] [Commented] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270222#comment-14270222 ] Jun Rao commented on KAFKA-1841: This is actually a bit tricky to fix. To make this really backward compatible, we have to make sure that version 0 of OffsetCommitRequest only writes to ZK. However, this doesn't quite work together with OffsetFetchRequest since in 0.8.2, it only has one version and it always reads offsets from Kafka. To address this issue, I bumped up the version of OffsetFetchRequest in 0.8.2 (with same wire protocol). Then, version 0 of OffsetFetchRequest will read from ZK and version 1 of OffsetFetchRequest will read from Kafka. This works as long as people are only using released final version. However, since this introduces an incompatible change of OffsetFetchRequest in 0.8.2-beta and trunk, this will create problems for people (assuming that they are using this api) who have a deployment of 0.8.2-beta and want to upgrade to 0.8.2 final, or a deployment from trunk and want to upgrade to a later version of trunk in the future. In either case, the upgrade of the broker will cause the old client to behave differently and incorrectly. OffsetCommitRequest API - timestamp field is not versioned -- Key: KAFKA-1841 URL: https://issues.apache.org/jira/browse/KAFKA-1841 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2 Environment: wire-protocol Reporter: Dana Powers Assignee: Jun Rao Priority: Blocker Fix For: 0.8.2 Attachments: kafka-1841.patch, kafka-1841_2015-01-08_15:07:57.patch Timestamp field was added to the OffsetCommitRequest wire protocol api for 0.8.2 by KAFKA-1012 . The 0.8.1.1 server does not support the timestamp field, so I think the api version of OffsetCommitRequest should be incremented and checked by the 0.8.2 kafka server before attempting to read a timestamp from the network buffer in OffsetCommitRequest.readFrom (core/src/main/scala/kafka/api/OffsetCommitRequest.scala) It looks like a subsequent patch (KAFKA-1462) added another api change to support a new constructor w/ params generationId and consumerId, calling that version 1, and a pending patch (KAFKA-1634) adds retentionMs as another field, while possibly removing timestamp altogether, calling this version 2. So the fix here is not straightforward enough for me to submit a patch. This could possibly be merged into KAFKA-1634, but opening as a separate Issue because I believe the lack of versioning in the current trunk should block 0.8.2 release. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer
Jun Rao created KAFKA-1849: -- Summary: Utils.readBytes() should support null ByteBuffer Key: KAFKA-1849 URL: https://issues.apache.org/jira/browse/KAFKA-1849 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.3 Reporter: Jun Rao Fix For: 0.8.3 We use Utils.readBytes to get the bytes from message.payload. However, message.payload can be null. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 28769: Patch for KAFKA-1809
On Jan. 7, 2015, 3:01 a.m., Jun Rao wrote: core/src/main/scala/kafka/cluster/Broker.scala, lines 36-52 https://reviews.apache.org/r/28769/diff/11/?file=807930#file807930line36 It would be good to document the format for both v1 and v2. Also, would it be cleaner if in v2, we remove the host/port fields and only include the endpoints field? Currently, v2 writes host/port, but doesn't use it itself. So, it's really just for backward compatibility. Since we have the use.new.wire.protocol config, we can let each broker register in ZK using v2 format only after that config is set to true (when all brokers are upgraded to the new binary). I think it makes sense to leave V1 as without endpoints and make V2 only endpoints, no host/port fields. We can use use.new.wire.protocol to choose which we serialize. This means that until we set new.protocol to true, non-plaintext endpoints will exist in config files (and therefore brokers will listen there), but will not exist in ZK. This is in line with our current understanding that until upgrade is finished (and use.new.protocol is set to true), non-plaintext endpoints will not be supported or used. A less expected side-effect: At the time we switch from V1 to V2, createEphemeralPathExpectConflictHandleZKBug will not recognize the existing broker (if ephemeral node is still around) as identical to the new registration (since the new broker will potentially have more endpoints). This means that if the ephemeral node is still around when we switch, we'll get broker already registered exception, instead of looping around until the ephemeral node goes away. I think we are fine with the behavior here, but I wanted to make it explicit. - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review66944 --- On Jan. 6, 2015, 7:46 p.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/ --- (Updated Jan. 6, 2015, 7:46 p.m.) Review request for kafka. Bugs: KAFKA-1809 https://issues.apache.org/jira/browse/KAFKA-1809 Repository: kafka Description --- first commit of refactoring. changed topicmetadata to include brokerendpoints and fixed few unit tests fixing systest and support for binding to default address fixed system tests fix default address binding and ipv6 support fix some issues regarding endpoint parsing. Also, larger segments for systest make the validation much faster added link to security wiki in doc fixing unit test after rename of ProtocolType to SecurityProtocol Following Joe's advice, added security protocol enum on client side, and modified protocol to use ID instead of string. validate producer config against enum add a second protocol for testing and modify SocketServerTests to check on multi-ports Reverted the metadata request changes and removed the explicit security protocol argument. Instead the socketserver will determine the protocol based on the port and add this to the request bump version for UpdateMetadataRequest and added support for rolling upgrades with new config following tests - fixed LeaderAndISR protocol and ZK registration for backward compatibility cleaned up some changes that were actually not necessary. hopefully making this patch slightly easier to review undoing some changes that don't belong here bring back config lost in cleanup fixes neccessary for an all non-plaintext cluster to work Diffs - clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 525b95e98010cd2053eacd8c321d079bcac2f910 clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java 527dd0f9c47fce7310b7a37a9b95bf87f1b9c292 clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java 6e37ea553f73d9c584641c48c56dbf6e62ba5f88 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java a39fab532f73148316a56c0f8e9197f38ea66f79 config/server.properties b0e4496a8ca736b6abe965a430e8ce87b0e8287f core/src/main/scala/kafka/admin/AdminUtils.scala 28b12c7b89a56c113b665fbde1b95f873f8624a3 core/src/main/scala/kafka/admin/TopicCommand.scala 285c0333ff43543d3e46444c1cd9374bb883bb59 core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala 84f60178f6ebae735c8aa3e79ed93fe21ac4aea7 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 4ff7e8f8cc695551dd5d2fe65c74f6b6c571e340 core/src/main/scala/kafka/api/TopicMetadata.scala
[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]}, {topic:testingTopic,partition:20,replicas:[3,1]},
Review Request 29751: Patch for kafka-1851
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29751/ --- Review request for kafka. Bugs: kafka-1851 https://issues.apache.org/jira/browse/kafka-1851 Repository: kafka Description --- fix the behavior of OffsetFetchRequest on unknown partition Diffs - core/src/main/scala/kafka/server/KafkaApis.scala 2f009920af37de3cf0a3eb131f2124f4e532c4e4 core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 8c5364fa97da1be09973c176d1baeb339455d319 Diff: https://reviews.apache.org/r/29751/diff/ Testing --- Thanks, Jun Rao
[jira] [Commented] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions
[ https://issues.apache.org/jira/browse/KAFKA-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270464#comment-14270464 ] Jun Rao commented on KAFKA-1851: Created reviewboard https://reviews.apache.org/r/29751/diff/ against branch origin/0.8.2 OffsetFetchRequest returns extra partitions when input only contains unknown partitions --- Key: KAFKA-1851 URL: https://issues.apache.org/jira/browse/KAFKA-1851 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Jun Rao Priority: Blocker Fix For: 0.8.2 Attachments: kafka-1851.patch When issuing an OffsetFetchRequest with an unknown topic partition, the OffsetFetchResponse unexpectedly returns all partitions in the same consumer group, in addition to the unknown partition. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1851) OffsetFetchRequest returns extra partitions when input only contains unknown partitions
[ https://issues.apache.org/jira/browse/KAFKA-1851?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1851: --- Status: Patch Available (was: Open) OffsetFetchRequest returns extra partitions when input only contains unknown partitions --- Key: KAFKA-1851 URL: https://issues.apache.org/jira/browse/KAFKA-1851 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Jun Rao Priority: Blocker Fix For: 0.8.2 Attachments: kafka-1851.patch When issuing an OffsetFetchRequest with an unknown topic partition, the OffsetFetchResponse unexpectedly returns all partitions in the same consumer group, in addition to the unknown partition. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29590: 1. Removed defaults for serializer/deserializer. 2. Converted type cast exception to serialization exception in the producer. 3. Added string ser/deser. 4. Moved the isKey fl
On Jan. 8, 2015, 12:07 p.m., Manikumar Reddy O wrote: clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java, line 34 https://reviews.apache.org/r/29590/diff/1/?file=806858#file806858line34 we should support null values right?. This is required for compaction.. That's a good point. Submitted a new RB with the change. - Jun --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29590/#review67194 --- On Jan. 5, 2015, 7:47 p.m., Jun Rao wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29590/ --- (Updated Jan. 5, 2015, 7:47 p.m.) Review request for kafka. Bugs: kafka-1797 https://issues.apache.org/jira/browse/kafka-1797 Repository: kafka Description --- addressing Jay's comments Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java 514cbd2c27a8d1ce13489d315f7880dfade7ffde clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 1d64f08762b0c33fcaebde0f41039b327060215a clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java c774a199db71fbc00776cd1256af57b2d9e55a66 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java fe9066388f4b7910512d85ef088a1b96749735ac clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java 9005b74a328c997663232fe3a0999b25d2267efe clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java d859fc588a276eb36bcfd621ae6d7978ad0decdd clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 9cdc13d6cbb372b350acf90a21538b8ba495d2e8 clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java de87f9c1caeadd176195be75d0db43fc0a518380 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java 3d4ab7228926f50309c07f0672f33416ce4fa37f clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java a5433398fb9788e260a4250da32e4be607f3f207 clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java PRE-CREATION core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala e194942492324092f811b86f9c1f28f79b366cfd core/src/main/scala/kafka/tools/ConsoleProducer.scala 397d80da08c925757649b7d104d8360f56c604c3 core/src/main/scala/kafka/tools/MirrorMaker.scala 2126f6e55c5ec6a418165d340cc9a4f445af5045 core/src/main/scala/kafka/tools/ProducerPerformance.scala f2dc4ed2f04f0e9656e10b02db5ed1d39c4a4d39 core/src/main/scala/kafka/tools/ReplayLogProducer.scala f541987b2876a438c43ea9088ae8fed708ba82a3 core/src/main/scala/kafka/tools/TestEndToEndLatency.scala 2ebc7bf643ea91bd93ba37b8e64e8a5a9bb37ece core/src/main/scala/kafka/tools/TestLogCleaning.scala b81010ec0fa9835bfe48ce6aad0c491cdc67e7ef core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala 1505fd4464dc9ac71cce52d9b64406a21e5e45d2 core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 6196060edf9f1650720ec916f88933953a1daa2c core/src/test/scala/unit/kafka/utils/TestUtils.scala 94d0028d8c4907e747aa8a74a13d719b974c97bf Diff: https://reviews.apache.org/r/29590/diff/ Testing --- Thanks, Jun Rao
[jira] [Commented] (KAFKA-1797) add the serializer/deserializer api to the new java client
[ https://issues.apache.org/jira/browse/KAFKA-1797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270349#comment-14270349 ] Jun Rao commented on KAFKA-1797: Created reviewboard https://reviews.apache.org/r/29738/diff/ against branch origin/0.8.2 add the serializer/deserializer api to the new java client -- Key: KAFKA-1797 URL: https://issues.apache.org/jira/browse/KAFKA-1797 Project: Kafka Issue Type: Improvement Components: core Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Jun Rao Fix For: 0.8.2 Attachments: kafka-1797.patch, kafka-1797.patch, kafka-1797.patch, kafka-1797.patch, kafka-1797_2014-12-09_18:48:33.patch, kafka-1797_2014-12-15_15:36:24.patch, kafka-1797_2014-12-17_09:47:45.patch Currently, the new java clients take a byte array for both the key and the value. While this api is simple, it pushes the serialization/deserialization logic into the application. This makes it hard to reason about what type of data flows through Kafka and also makes it hard to share an implementation of the serializer/deserializer. For example, to support Avro, the serialization logic could be quite involved since it might need to register the Avro schema in some remote registry and maintain a schema cache locally, etc. Without a serialization api, it's impossible to share such an implementation so that people can easily reuse. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 29738: Patch for kafka-1797
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29738/ --- Review request for kafka. Bugs: kafka-1797 https://issues.apache.org/jira/browse/kafka-1797 Repository: kafka Description --- add null support in string serializer and deserializer Diffs - clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java a3b3700a1e0716643761d7032bd32bce839d3065 clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java 02db47f8736988343dd293fc3da03751f78a3b5c clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java d550a3137c066abb5e2984ac6245574832929ff8 Diff: https://reviews.apache.org/r/29738/diff/ Testing --- Thanks, Jun Rao
[jira] [Updated] (KAFKA-1797) add the serializer/deserializer api to the new java client
[ https://issues.apache.org/jira/browse/KAFKA-1797?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1797: --- Attachment: kafka-1797.patch add the serializer/deserializer api to the new java client -- Key: KAFKA-1797 URL: https://issues.apache.org/jira/browse/KAFKA-1797 Project: Kafka Issue Type: Improvement Components: core Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Jun Rao Fix For: 0.8.2 Attachments: kafka-1797.patch, kafka-1797.patch, kafka-1797.patch, kafka-1797.patch, kafka-1797_2014-12-09_18:48:33.patch, kafka-1797_2014-12-15_15:36:24.patch, kafka-1797_2014-12-17_09:47:45.patch Currently, the new java clients take a byte array for both the key and the value. While this api is simple, it pushes the serialization/deserialization logic into the application. This makes it hard to reason about what type of data flows through Kafka and also makes it hard to share an implementation of the serializer/deserializer. For example, to support Avro, the serialization logic could be quite involved since it might need to register the Avro schema in some remote registry and maintain a schema cache locally, etc. Without a serialization api, it's impossible to share such an implementation so that people can easily reuse. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1849. Resolution: Not a Problem Ok, we actually handle null explicitly in MessageAndMetadata. So, this is not an issue. Utils.readBytes() should support null ByteBuffer Key: KAFKA-1849 URL: https://issues.apache.org/jira/browse/KAFKA-1849 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.3 Reporter: Jun Rao Labels: newbie Fix For: 0.8.3 We use Utils.readBytes to get the bytes from message.payload. However, message.payload can be null. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic
Jun Rao created KAFKA-1852: -- Summary: OffsetCommitRequest can commit offset on unknown topic Key: KAFKA-1852 URL: https://issues.apache.org/jira/browse/KAFKA-1852 Project: Kafka Issue Type: Bug Affects Versions: 0.8.3 Reporter: Jun Rao Currently, we allow an offset to be committed to Kafka, even when the topic/partition for the offset doesn't exist. We probably should disallow that and send an error back in that case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]}, {topic:testingTopic,partition:20,replicas:[3,1]},
[jira] [Commented] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270447#comment-14270447 ] Jun Rao commented on KAFKA-1849: The issue is that MessageAndMetadata uses Utils.readBytes() to convert ByteBuffer to bytes for the key and payload, which can be null. Utils.readBytes() should support null ByteBuffer Key: KAFKA-1849 URL: https://issues.apache.org/jira/browse/KAFKA-1849 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.3 Reporter: Jun Rao Labels: newbie Fix For: 0.8.3 We use Utils.readBytes to get the bytes from message.payload. However, message.payload can be null. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1850) Failed reassignment leads to additional replica
Alex Tian created KAFKA-1850: Summary: Failed reassignment leads to additional replica Key: KAFKA-1850 URL: https://issues.apache.org/jira/browse/KAFKA-1850 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Environment: CentOS (Linux Kernel 2.6.32-71.el6.x86_64 ) Reporter: Alex Tian Assignee: Neha Narkhede Priority: Minor When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: ## alex-topics-to-move.json {topics: [{topic: testingTopic}], version:1 } ## Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} ## Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]},
[jira] [Updated] (KAFKA-1836) metadata.fetch.timeout.ms set to zero blocks forever
[ https://issues.apache.org/jira/browse/KAFKA-1836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1836: Attachment: KAFKA-1836-new-patch.patch metadata.fetch.timeout.ms set to zero blocks forever Key: KAFKA-1836 URL: https://issues.apache.org/jira/browse/KAFKA-1836 Project: Kafka Issue Type: Bug Components: clients Affects Versions: 0.8.2 Reporter: Paul Pearcy Priority: Minor Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1836-new-patch.patch, KAFKA-1836.patch You can easily work around this by setting the timeout value to 1ms, but 0ms should mean 0ms or at least have the behavior documented. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1836) metadata.fetch.timeout.ms set to zero blocks forever
[ https://issues.apache.org/jira/browse/KAFKA-1836?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270525#comment-14270525 ] jaikiran pai commented on KAFKA-1836: - Thanks Ewen for your feedback. I've taken into account your comments and updated the patch accordingly and created a review request https://reviews.apache.org/r/29752/. I've also uploaded the same patch here to the JIRA. metadata.fetch.timeout.ms set to zero blocks forever Key: KAFKA-1836 URL: https://issues.apache.org/jira/browse/KAFKA-1836 Project: Kafka Issue Type: Bug Components: clients Affects Versions: 0.8.2 Reporter: Paul Pearcy Priority: Minor Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1836-new-patch.patch, KAFKA-1836.patch You can easily work around this by setting the timeout value to 1ms, but 0ms should mean 0ms or at least have the behavior documented. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1853: Status: Patch Available (was: Open) Created reviewboard https://reviews.apache.org/r/29755/diff against branch origin/trunk Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/KAFKA-1853 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Reporter: jaikiran pai Fix For: 0.8.3 As noted in this discussion in the user mailing list http://mail-archives.apache.org/mod_mbox/kafka-users/201501.mbox/%3C54AE3661.8080007%40gmail.com%3E an unsuccessful attempt at renaming the underlying files of a LogSegment can lead to file leaks and also leave the LogSegment in an invalid state. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission
jaikiran pai created KAFKA-1854: --- Summary: Allow the JIRA username and password to be prompted during patch submission Key: KAFKA-1854 URL: https://issues.apache.org/jira/browse/KAFKA-1854 Project: Kafka Issue Type: Improvement Reporter: jaikiran pai The current patch submission process involves using the kafka-patch-review.py python script which expects a jira.ini file to contain the user's username and password for JIRA authentication. I'm one of those who doesn't like storing passwords in files :) It would be good to (optionally) allow the username/password to be prompted by the patch submission script. I've a patch which I can submit for this enhancement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission
[ https://issues.apache.org/jira/browse/KAFKA-1854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1854: Attachment: KAFKA-1854.patch Allow the JIRA username and password to be prompted during patch submission --- Key: KAFKA-1854 URL: https://issues.apache.org/jira/browse/KAFKA-1854 Project: Kafka Issue Type: Improvement Reporter: jaikiran pai Attachments: KAFKA-1854.patch The current patch submission process involves using the kafka-patch-review.py python script which expects a jira.ini file to contain the user's username and password for JIRA authentication. I'm one of those who doesn't like storing passwords in files :) It would be good to (optionally) allow the username/password to be prompted by the patch submission script. I've a patch which I can submit for this enhancement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 29756: Patch for KAFKA-1854
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29756/ --- Review request for kafka. Bugs: KAFKA-1854 https://issues.apache.org/jira/browse/KAFKA-1854 Repository: kafka Description --- KAFKA-1854 Allow JIRA username and password to be prompted in the absence of a jira.ini file, during patch submission Diffs - kafka-patch-review.py b7f132f9d210b8648859ab8f9c89f30ec128ab38 Diff: https://reviews.apache.org/r/29756/diff/ Testing --- Thanks, Jaikiran Pai
[jira] [Commented] (KAFKA-1854) Allow the JIRA username and password to be prompted during patch submission
[ https://issues.apache.org/jira/browse/KAFKA-1854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270717#comment-14270717 ] jaikiran pai commented on KAFKA-1854: - Created reviewboard https://reviews.apache.org/r/29756/diff/ against branch origin/trunk Allow the JIRA username and password to be prompted during patch submission --- Key: KAFKA-1854 URL: https://issues.apache.org/jira/browse/KAFKA-1854 Project: Kafka Issue Type: Improvement Reporter: jaikiran pai Attachments: KAFKA-1854.patch The current patch submission process involves using the kafka-patch-review.py python script which expects a jira.ini file to contain the user's username and password for JIRA authentication. I'm one of those who doesn't like storing passwords in files :) It would be good to (optionally) allow the username/password to be prompted by the patch submission script. I've a patch which I can submit for this enhancement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270551#comment-14270551 ] Joe Stein commented on KAFKA-1850: -- An attachment will make it easier for folks to look at your outputs and/or logs. What version of zookeeper are you using? What distribution? Do you know what broker is the controller? Check out all of the logs on the broker that is the controller. Look for errors. Upload those too. Failed reassignment leads to additional replica --- Key: KAFKA-1850 URL: https://issues.apache.org/jira/browse/KAFKA-1850 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Environment: CentOS (Linux Kernel 2.6.32-71.el6.x86_64 ) Reporter: Alex Tian Assignee: Neha Narkhede Priority: Minor Labels: newbie Original Estimate: 504h Remaining Estimate: 504h When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]},
Re: Review Request 29755: Patch for KAFKA-1853
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29755/#review67382 --- This is my very first attempt at Scala, so feel free to let me know if anything looks incorrect. I've added a test to ensure that the change at least works as per my expectation. - Jaikiran Pai On Jan. 9, 2015, 6:44 a.m., Jaikiran Pai wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29755/ --- (Updated Jan. 9, 2015, 6:44 a.m.) Review request for kafka. Bugs: KAFKA-1853 https://issues.apache.org/jira/browse/KAFKA-1853 Repository: kafka Description --- Fixes file leaks caused by unsuccessful file rename attempt in LogSegment Diffs - core/src/main/scala/kafka/log/FileMessageSet.scala b2652ddbe2f857028d5980e29a008b2c614694a3 core/src/test/scala/unit/kafka/log/LogSegmentTest.scala 03fb3512c4a4450eac83d4cd4b0919baeaa22942 Diff: https://reviews.apache.org/r/29755/diff/ Testing --- Thanks, Jaikiran Pai
Re: Follow-up On Important Issues for 0.8.2
Adding User Community to see if any one knows behavior of Producer for issue #1) and status of 2). Thanks, Bhavesh On Fri, Jan 2, 2015 at 12:37 PM, Bhavesh Mistry mistry.p.bhav...@gmail.com wrote: Hi Kafka Dev Team, I am following-up with you guys regarding New (Java) Producer behavior in event of network or firewall rules. I just wanted to make Java Producer resilient of any network or firewall issues, and does not become single-point of failure in application: 1) Jira Issue https://issues.apache.org/jira/browse/KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788?focusedCommentId=14259235page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14259235 What should be the behavior of the Producer when it can not reach leader broker, but metadata reported broker is leader for that partition (via other broker) ? Should the record-error-rate be counted and Call Back should be called with error or not ? 1) *record-error-rate* metric remain zero despite following firewall rule. In my opinion, it should have called org.apache.kafka.clients.producer.Callback but I did not see that happening either one or two are brokers not reachable. 2) Is jira ticket https://issues.apache.org/jira/browse/KAFKA-1788 will be merged to 0.8.2 ? This will give the ability to close the producer in event of lost connectivity to broker if io thread misbehave (does not end) ? Thanks for your help ! Thanks, Bhavesh
[jira] [Updated] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joe Stein updated KAFKA-1853: - Fix Version/s: 0.8.3 Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/KAFKA-1853 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Reporter: jaikiran pai Fix For: 0.8.3 As noted in this discussion in the user mailing list http://mail-archives.apache.org/mod_mbox/kafka-users/201501.mbox/%3C54AE3661.8080007%40gmail.com%3E an unsuccessful attempt at renaming the underlying files of a LogSegment can lead to file leaks and also leave the LogSegment in an invalid state. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[DISCUSS] 0.8.2-beta2 release
Hi, I was thinking that once all the blockers are committed for 0.8.2 that we should release beta2. Thoughts? /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop /
Re: Review Request 29738: Patch for kafka-1797
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29738/#review67374 --- Ship it! Ship It! - Manikumar Reddy O On Jan. 9, 2015, 1:22 a.m., Jun Rao wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29738/ --- (Updated Jan. 9, 2015, 1:22 a.m.) Review request for kafka. Bugs: kafka-1797 https://issues.apache.org/jira/browse/kafka-1797 Repository: kafka Description --- add null support in string serializer and deserializer Diffs - clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java a3b3700a1e0716643761d7032bd32bce839d3065 clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java 02db47f8736988343dd293fc3da03751f78a3b5c clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java d550a3137c066abb5e2984ac6245574832929ff8 Diff: https://reviews.apache.org/r/29738/diff/ Testing --- Thanks, Jun Rao
[jira] [Updated] (KAFKA-1853) Unsuccessful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] jaikiran pai updated KAFKA-1853: Summary: Unsuccessful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state (was: Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state) Unsuccessful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state - Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/KAFKA-1853 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Reporter: jaikiran pai Fix For: 0.8.3 As noted in this discussion in the user mailing list http://mail-archives.apache.org/mod_mbox/kafka-users/201501.mbox/%3C54AE3661.8080007%40gmail.com%3E an unsuccessful attempt at renaming the underlying files of a LogSegment can lead to file leaks and also leave the LogSegment in an invalid state. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29724: Patch for KAFKA-1566
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/#review67376 --- Ship it! Ship It! - Manikumar Reddy O On Jan. 8, 2015, 8:47 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/ --- (Updated Jan. 8, 2015, 8:47 p.m.) Review request for kafka. Bugs: KAFKA-1566 https://issues.apache.org/jira/browse/KAFKA-1566 Repository: kafka Description --- KAFKA-1566. Kafka environment configuration (kafka-env.sh). Diffs - bin/kafka-run-class.sh ce3a4d06a27f66a33d729a15207b2d226b100c6a bin/windows/kafka-run-class.bat 9df3d2b45236b4f06d55a89c84afcf0ab9f5d0f2 config/kafka-env.cmd PRE-CREATION config/kafka-env.sh PRE-CREATION Diff: https://reviews.apache.org/r/29724/diff/ Testing --- Thanks, Sriharsha Chintalapani
[jira] [Created] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state
jaikiran pai created KAFKA-1853: --- Summary: Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/KAFKA-1853 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Reporter: jaikiran pai As noted in this discussion in the user mailing list http://mail-archives.apache.org/mod_mbox/kafka-users/201501.mbox/%3C54AE3661.8080007%40gmail.com%3E an unsuccessful attempt at renaming the underlying files of a LogSegment can lead to file leaks and also leave the LogSegment in an invalid state. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1853) Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state
[ https://issues.apache.org/jira/browse/KAFKA-1853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270627#comment-14270627 ] jaikiran pai commented on KAFKA-1853: - I've a patch for this which I'll submit in a while. Unsuccesful suffix rename attempt of LogSegment can leak files and also leave the LogSegment in an invalid state Key: KAFKA-1853 URL: https://issues.apache.org/jira/browse/KAFKA-1853 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Reporter: jaikiran pai As noted in this discussion in the user mailing list http://mail-archives.apache.org/mod_mbox/kafka-users/201501.mbox/%3C54AE3661.8080007%40gmail.com%3E an unsuccessful attempt at renaming the underlying files of a LogSegment can lead to file leaks and also leave the LogSegment in an invalid state. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 29755: Patch for KAFKA-1853
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29755/ --- Review request for kafka. Bugs: KAFKA-1853 https://issues.apache.org/jira/browse/KAFKA-1853 Repository: kafka Description --- Fixes file leaks caused by unsuccessful file rename attempt in LogSegment Diffs - core/src/main/scala/kafka/log/FileMessageSet.scala b2652ddbe2f857028d5980e29a008b2c614694a3 core/src/test/scala/unit/kafka/log/LogSegmentTest.scala 03fb3512c4a4450eac83d4cd4b0919baeaa22942 Diff: https://reviews.apache.org/r/29755/diff/ Testing --- Thanks, Jaikiran Pai
Re: Review Request 29756: Patch for KAFKA-1854
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29756/#review67388 --- This one needs a minor change which I'm going to submit as an update - Jaikiran Pai On Jan. 9, 2015, 7:50 a.m., Jaikiran Pai wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29756/ --- (Updated Jan. 9, 2015, 7:50 a.m.) Review request for kafka. Bugs: KAFKA-1854 https://issues.apache.org/jira/browse/KAFKA-1854 Repository: kafka Description --- KAFKA-1854 Allow JIRA username and password to be prompted in the absence of a jira.ini file, during patch submission Diffs - kafka-patch-review.py b7f132f9d210b8648859ab8f9c89f30ec128ab38 Diff: https://reviews.apache.org/r/29756/diff/ Testing --- Thanks, Jaikiran Pai
[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270430#comment-14270430 ] Alex Tian commented on KAFKA-1850: -- Maybe my reassignment plan is not the best choice: At first I have 3 brokers: 0, 1 and 2. Then I expanded the cluster to 5 brokers, adding Broker 3 and Broker 4. The topic testingTopic resides in 0, 1 and 2, but I reassign it to 0, 1, 2, 3 and 4 since the topic is too heavy for the original 3 brokers. Failed reassignment leads to additional replica --- Key: KAFKA-1850 URL: https://issues.apache.org/jira/browse/KAFKA-1850 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Environment: CentOS (Linux Kernel 2.6.32-71.el6.x86_64 ) Reporter: Alex Tian Assignee: Neha Narkhede Priority: Minor Labels: newbie Original Estimate: 504h Remaining Estimate: 504h When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: ## alex-topics-to-move.json {topics: [{topic: testingTopic}], version:1 } ## Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} ## Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]},
[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]}, {topic:testingTopic,partition:20,replicas:[3,1]},
[jira] [Commented] (KAFKA-1849) Utils.readBytes() should support null ByteBuffer
[ https://issues.apache.org/jira/browse/KAFKA-1849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270385#comment-14270385 ] Jay Kreps commented on KAFKA-1849: -- Not sure if that is the right place for it. I think Utils.readBytes is a generic ByteBuffer=byte[] utility. It doesn't deal with the kafka-specific encoding of null=bytes nor does it handle the serialized array size. Utils.readBytes() should support null ByteBuffer Key: KAFKA-1849 URL: https://issues.apache.org/jira/browse/KAFKA-1849 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.3 Reporter: Jun Rao Labels: newbie Fix For: 0.8.3 We use Utils.readBytes to get the bytes from message.payload. However, message.payload can be null. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Tian updated KAFKA-1850: - Description: When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]}, {topic:testingTopic,partition:32,replicas:[0,1]}, {topic:testingTopic,partition:18,replicas:[1,3]}, {topic:testingTopic,partition:31,replicas:[4,0]}, {topic:testingTopic,partition:23,replicas:[1,4]}, {topic:testingTopic,partition:9,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[2,4]}, {topic:testingTopic,partition:34,replicas:[2,3]}, {topic:testingTopic,partition:17,replicas:[0,2]}, {topic:testingTopic,partition:20,replicas:[3,1]},
[jira] [Commented] (KAFKA-1850) Failed reassignment leads to additional replica
[ https://issues.apache.org/jira/browse/KAFKA-1850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270451#comment-14270451 ] Alex Tian commented on KAFKA-1850: -- Sorry again for too long change history here, maybe I should have attached a file specifying all the output. Failed reassignment leads to additional replica --- Key: KAFKA-1850 URL: https://issues.apache.org/jira/browse/KAFKA-1850 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.1 Environment: CentOS (Linux Kernel 2.6.32-71.el6.x86_64 ) Reporter: Alex Tian Assignee: Neha Narkhede Priority: Minor Labels: newbie Original Estimate: 504h Remaining Estimate: 504h When I start a topic reassignment (Totally 36 partitions) in my Kafka Cluster, 24 partitions succeeded and 12 ones failed. However, the 12 failed partitions have more replicas. I think the reason is that AR still consists of RAR and OAR although the reassignment for the partition failed. Could we regard this problem as a bug? Quite sorry if any mistake in my question, since I am a beginner for Kafka. This is the output from operation: 1. alex-topics-to-move.json: {topics: [{topic: testingTopic}], version:1 } 2. Generate a reassignment plan $./kafka-reassign-partitions.sh --generate --broker-list 0,1,2,3,4 --topics-to-move-json-file ./alex-topics-to-move.json --zookeeper 192.168.112.95:2181,192.168.112.96:2181,192.168.112.97:2181,192.168.112.98:2181,192.168.112.99:2181 Current partition replica assignment {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,2]}, {topic:testingTopic,partition:1,replicas:[1,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[0,1]}, {topic:testingTopic,partition:16,replicas:[1,0]}, {topic:testingTopic,partition:32,replicas:[2,0]}, {topic:testingTopic,partition:18,replicas:[0,1]}, {topic:testingTopic,partition:31,replicas:[1,2]}, {topic:testingTopic,partition:9,replicas:[0,2]}, {topic:testingTopic,partition:23,replicas:[2,1]}, {topic:testingTopic,partition:19,replicas:[1,2]}, {topic:testingTopic,partition:34,replicas:[1,0]}, {topic:testingTopic,partition:17,replicas:[2,1]}, {topic:testingTopic,partition:7,replicas:[1,2]}, {topic:testingTopic,partition:20,replicas:[2,0]}, {topic:testingTopic,partition:8,replicas:[2,0]}, {topic:testingTopic,partition:11,replicas:[2,1]}, {topic:testingTopic,partition:3,replicas:[0,2]}, {topic:testingTopic,partition:30,replicas:[0,1]}, {topic:testingTopic,partition:35,replicas:[2,1]}, {topic:testingTopic,partition:26,replicas:[2,0]}, {topic:testingTopic,partition:22,replicas:[1,0]}, {topic:testingTopic,partition:10,replicas:[1,0]}, {topic:testingTopic,partition:24,replicas:[0,1]}, {topic:testingTopic,partition:21,replicas:[0,2]}, {topic:testingTopic,partition:15,replicas:[0,2]}, {topic:testingTopic,partition:4,replicas:[1,0]}, {topic:testingTopic,partition:28,replicas:[1,0]}, {topic:testingTopic,partition:25,replicas:[1,2]},: {topic:testingTopic,partition:14,replicas:[2,0]}, {topic:testingTopic,partition:2,replicas:[2,0]}, {topic:testingTopic,partition:13,replicas:[1,2]}, {topic:testingTopic,partition:5,replicas:[2,1]}, {topic:testingTopic,partition:29,replicas:[2,1]}, {topic:testingTopic,partition:33,replicas:[0,2]}, {topic:testingTopic,partition:0,replicas:[0,1]}]} Proposed partition reassignment configuration ( alex-expand-cluster-reassignment.json ) {version:1, partitions:[ {topic:testingTopic,partition:27,replicas:[0,4]}, {topic:testingTopic,partition:1,replicas:[4,2]}, {topic:testingTopic,partition:12,replicas:[0,1]}, {topic:testingTopic,partition:6,replicas:[4,3]}, {topic:testingTopic,partition:16,replicas:[4,1]},
Re: Review Request 29523: Patch for KAFKA-1723
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/#review67314 --- clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java https://reviews.apache.org/r/29523/#comment111310 This won't be enough to distinguish the metric, right? Absent the group/tags info... - Jay Kreps On Jan. 8, 2015, 4:34 p.m., Manikumar Reddy O wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:34 p.m.) Review request for kafka. Bugs: KAFKA-1723 https://issues.apache.org/jira/browse/KAFKA-1723 Repository: kafka Description --- Standard JMX MBean Naming is implemented;Addresing Jay's comments Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 1bce50185273dbdbc131fbc9c7f5f3e9c346517a clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 7f8a41c4bf437711685a8271a4d3c83a176dd957 clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 8cab16c0a0bdb671fea1fc2fc2694247f66cc971 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 3053f2745c8e5f6e3b75826d3749656f150878db clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java aa91e1444a49c55870b9a7a32086fa2b04471fba clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 84a7a07269c51ccc22ebb4ff9797292d07ba778e clients/src/main/java/org/apache/kafka/common/Metric.java b023e8e7c486adf21ed9a554085ab8ad7f3ee038 clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java 29185a6a90d0035d650c7e56ce612a0878cb115c clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 3c312011a7ff7e79c277a89048e7e62ebd6078db clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java a7458b50cb16fbb2b31b857d5b359e65258bbf08 clients/src/main/java/org/apache/kafka/common/metrics/MetricName.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java 49be4019ac03835701c49646920766228ac7ffe9 clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java 25c1faf2887ea02708c1f5b5f822f5299ed86bd6 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java 7365ceb39072a6a1ecf533f5a20830ed1f2cfc72 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java c70d577ada8c099533d4f4ed2e86d37e0a6e6676 clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java fe3c13f319d48b89a4f26b6d78c2c3e31cc50d7e clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java 2f43c49450e1a3d671bd17417dc42941f1858750 clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java 19bea0f1fa1ebf15d86623015ec909b0155e4bd3 clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 clients/src/test/java/org/apache/kafka/test/MetricsBench.java 9d98c1148255455fd801043b59b98fed9d0b76b3 Diff: https://reviews.apache.org/r/29523/diff/ Testing --- Thanks, Manikumar Reddy O
[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270163#comment-14270163 ] Jay Kreps commented on KAFKA-1723: -- 2. I think you are saying that sensor names are still strings not compounds. I think this makes sense according to your proposal. 3. We really can't make client id mandatory as most people don't set it and forcing them will be quite annoying. We can definitely set some default like producer, though. There is no guarantee that client id is unique, of course. But people who have multiple producers can set different client ids to distinguish them. Another question: How are you supposed to make use of the .metrics() api on the producer and consumer now? Previously the idea was that you could do something like {code} if(producer.metrics.get(my.metric.name) MAXIMUM) alert(This is bad!); {code} But now this use case is a bit mixed up, right? As now the full name is an object which is a bit hard to construct. So would you do something like: {code} MapString, String tags = new HashMapString, String(); tags.put(clientId, my-id); MetricName name = new MetricName(my.metric.name, my-group, tags); if(producer.metrics.get(my.metric.name) MAXIMUM) alert(This is bad!); {code} This is a bit awkward, right? Is there anything we can do to simplify usage? Also, in this patch the metric names are lower case and dash separated but the metric groups and tag names are camel case. That seems pretty odd. Perhaps we could make them all lower case and dash separated? E.g. new MetricName(record-send-rate, producer-metrics, Map(client-id, foo)) make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch, KAFKA-1723_2015-01-08_22:02:22.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 29728: Patch for KAFKA-1848
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29728/ --- Review request for kafka. Bugs: KAFKA-1848 https://issues.apache.org/jira/browse/KAFKA-1848 Repository: kafka Description --- Locally commit minor fix Diffs - core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 191a8677444e53b043e9ad6e94c5a9191c32599e Diff: https://reviews.apache.org/r/29728/diff/ Testing --- Thanks, Aditya Auradkar
Re: Review Request 29728: Patch for KAFKA-1848
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29728/ --- (Updated Jan. 8, 2015, 10:49 p.m.) Review request for kafka. Bugs: KAFKA-1848 https://issues.apache.org/jira/browse/KAFKA-1848 Repository: kafka Description (updated) --- Fix for KAFKA-1848. Diffs - core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 191a8677444e53b043e9ad6e94c5a9191c32599e Diff: https://reviews.apache.org/r/29728/diff/ Testing --- Thanks, Aditya Auradkar
Re: Review Request 29692: Patch for kafka-1841
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29692/ --- (Updated Jan. 8, 2015, 11:07 p.m.) Review request for kafka. Bugs: kafka-1841 https://issues.apache.org/jira/browse/kafka-1841 Repository: kafka Description (updated) --- version 0 of fetchOffset reads from ZK and version 1 of fetchOffset reads from Kafka Diffs (updated) - clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java 7517b879866fc5dad5f8d8ad30636da8bbe7784a clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java 3ee5cbad55ce836fd04bb954dcf6ef2f9bc3288f core/src/main/scala/kafka/api/OffsetCommitRequest.scala 861a6cf11dc6b6431fcbbe9de00c74a122f204bd core/src/main/scala/kafka/api/OffsetFetchRequest.scala c7604b9cdeb8f46507f04ed7d35fcb3d5f150713 core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 4cabffeacea09a49913505db19a96a55d58c0909 core/src/main/scala/kafka/server/KafkaApis.scala 2f009920af37de3cf0a3eb131f2124f4e532c4e4 core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala cd16ced5465d098be7a60498326b2a98c248f343 Diff: https://reviews.apache.org/r/29692/diff/ Testing --- Thanks, Jun Rao
[jira] [Commented] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270187#comment-14270187 ] Jun Rao commented on KAFKA-1841: Updated reviewboard https://reviews.apache.org/r/29692/diff/ against branch origin/0.8.2 OffsetCommitRequest API - timestamp field is not versioned -- Key: KAFKA-1841 URL: https://issues.apache.org/jira/browse/KAFKA-1841 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2 Environment: wire-protocol Reporter: Dana Powers Assignee: Jun Rao Priority: Blocker Fix For: 0.8.2 Attachments: kafka-1841.patch, kafka-1841_2015-01-08_15:07:57.patch Timestamp field was added to the OffsetCommitRequest wire protocol api for 0.8.2 by KAFKA-1012 . The 0.8.1.1 server does not support the timestamp field, so I think the api version of OffsetCommitRequest should be incremented and checked by the 0.8.2 kafka server before attempting to read a timestamp from the network buffer in OffsetCommitRequest.readFrom (core/src/main/scala/kafka/api/OffsetCommitRequest.scala) It looks like a subsequent patch (KAFKA-1462) added another api change to support a new constructor w/ params generationId and consumerId, calling that version 1, and a pending patch (KAFKA-1634) adds retentionMs as another field, while possibly removing timestamp altogether, calling this version 2. So the fix here is not straightforward enough for me to submit a patch. This could possibly be merged into KAFKA-1634, but opening as a separate Issue because I believe the lack of versioning in the current trunk should block 0.8.2 release. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1841) OffsetCommitRequest API - timestamp field is not versioned
[ https://issues.apache.org/jira/browse/KAFKA-1841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1841: --- Attachment: kafka-1841_2015-01-08_15:07:57.patch OffsetCommitRequest API - timestamp field is not versioned -- Key: KAFKA-1841 URL: https://issues.apache.org/jira/browse/KAFKA-1841 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2 Environment: wire-protocol Reporter: Dana Powers Assignee: Jun Rao Priority: Blocker Fix For: 0.8.2 Attachments: kafka-1841.patch, kafka-1841_2015-01-08_15:07:57.patch Timestamp field was added to the OffsetCommitRequest wire protocol api for 0.8.2 by KAFKA-1012 . The 0.8.1.1 server does not support the timestamp field, so I think the api version of OffsetCommitRequest should be incremented and checked by the 0.8.2 kafka server before attempting to read a timestamp from the network buffer in OffsetCommitRequest.readFrom (core/src/main/scala/kafka/api/OffsetCommitRequest.scala) It looks like a subsequent patch (KAFKA-1462) added another api change to support a new constructor w/ params generationId and consumerId, calling that version 1, and a pending patch (KAFKA-1634) adds retentionMs as another field, while possibly removing timestamp altogether, calling this version 2. So the fix here is not straightforward enough for me to submit a patch. This could possibly be merged into KAFKA-1634, but opening as a separate Issue because I believe the lack of versioning in the current trunk should block 0.8.2 release. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29590: 1. Removed defaults for serializer/deserializer. 2. Converted type cast exception to serialization exception in the producer. 3. Added string ser/deser. 4. Moved the isKey fl
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29590/#review67194 --- clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java https://reviews.apache.org/r/29590/#comment111066 we should support null values right?. This is required for compaction.. - Manikumar Reddy O On Jan. 5, 2015, 7:47 p.m., Jun Rao wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29590/ --- (Updated Jan. 5, 2015, 7:47 p.m.) Review request for kafka. Bugs: kafka-1797 https://issues.apache.org/jira/browse/kafka-1797 Repository: kafka Description --- addressing Jay's comments Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java 514cbd2c27a8d1ce13489d315f7880dfade7ffde clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 1d64f08762b0c33fcaebde0f41039b327060215a clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java c774a199db71fbc00776cd1256af57b2d9e55a66 clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java fe9066388f4b7910512d85ef088a1b96749735ac clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java 9005b74a328c997663232fe3a0999b25d2267efe clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java d859fc588a276eb36bcfd621ae6d7978ad0decdd clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 9cdc13d6cbb372b350acf90a21538b8ba495d2e8 clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java de87f9c1caeadd176195be75d0db43fc0a518380 clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java 3d4ab7228926f50309c07f0672f33416ce4fa37f clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java a5433398fb9788e260a4250da32e4be607f3f207 clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java PRE-CREATION core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala e194942492324092f811b86f9c1f28f79b366cfd core/src/main/scala/kafka/tools/ConsoleProducer.scala 397d80da08c925757649b7d104d8360f56c604c3 core/src/main/scala/kafka/tools/MirrorMaker.scala 2126f6e55c5ec6a418165d340cc9a4f445af5045 core/src/main/scala/kafka/tools/ProducerPerformance.scala f2dc4ed2f04f0e9656e10b02db5ed1d39c4a4d39 core/src/main/scala/kafka/tools/ReplayLogProducer.scala f541987b2876a438c43ea9088ae8fed708ba82a3 core/src/main/scala/kafka/tools/TestEndToEndLatency.scala 2ebc7bf643ea91bd93ba37b8e64e8a5a9bb37ece core/src/main/scala/kafka/tools/TestLogCleaning.scala b81010ec0fa9835bfe48ce6aad0c491cdc67e7ef core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala 1505fd4464dc9ac71cce52d9b64406a21e5e45d2 core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 6196060edf9f1650720ec916f88933953a1daa2c core/src/test/scala/unit/kafka/utils/TestUtils.scala 94d0028d8c4907e747aa8a74a13d719b974c97bf Diff: https://reviews.apache.org/r/29590/diff/ Testing --- Thanks, Jun Rao
unable to sign up for kafka email distribution
Unable to confirm myself on kafka email list. Mail is getting kicked back from apache email servers according to our tech department. Thanks!
[jira] [Commented] (KAFKA-1845) KafkaConfig should use ConfigDef
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269348#comment-14269348 ] Andrii Biletskyi commented on KAFKA-1845: - I converted it to sub-task and put under CLI tool since we will require this feature in GlobalConfiguration task (namely validating ConfigChangeRequest). KafkaConfig should use ConfigDef - Key: KAFKA-1845 URL: https://issues.apache.org/jira/browse/KAFKA-1845 Project: Kafka Issue Type: Sub-task Reporter: Gwen Shapira Labels: newbie ConfigDef is already used for the new producer and for TopicConfig. Will be nice to standardize and use one configuration and validation library across the board. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1723: --- Attachment: KAFKA-1723_2015-01-08_22:02:22.patch make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch, KAFKA-1723_2015-01-08_22:02:22.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29523: Patch for KAFKA-1723
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:34 p.m.) Review request for kafka. Bugs: KAFKA-1723 https://issues.apache.org/jira/browse/KAFKA-1723 Repository: kafka Description --- Standard JMX MBean Naming is implemented;Addresing Jay's comments Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 1bce50185273dbdbc131fbc9c7f5f3e9c346517a clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 7f8a41c4bf437711685a8271a4d3c83a176dd957 clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 8cab16c0a0bdb671fea1fc2fc2694247f66cc971 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 3053f2745c8e5f6e3b75826d3749656f150878db clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java aa91e1444a49c55870b9a7a32086fa2b04471fba clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 84a7a07269c51ccc22ebb4ff9797292d07ba778e clients/src/main/java/org/apache/kafka/common/Metric.java b023e8e7c486adf21ed9a554085ab8ad7f3ee038 clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java 29185a6a90d0035d650c7e56ce612a0878cb115c clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 3c312011a7ff7e79c277a89048e7e62ebd6078db clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java a7458b50cb16fbb2b31b857d5b359e65258bbf08 clients/src/main/java/org/apache/kafka/common/metrics/MetricName.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java 49be4019ac03835701c49646920766228ac7ffe9 clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java 25c1faf2887ea02708c1f5b5f822f5299ed86bd6 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java 7365ceb39072a6a1ecf533f5a20830ed1f2cfc72 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java c70d577ada8c099533d4f4ed2e86d37e0a6e6676 clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java fe3c13f319d48b89a4f26b6d78c2c3e31cc50d7e clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java 2f43c49450e1a3d671bd17417dc42941f1858750 clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java 19bea0f1fa1ebf15d86623015ec909b0155e4bd3 clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 clients/src/test/java/org/apache/kafka/test/MetricsBench.java 9d98c1148255455fd801043b59b98fed9d0b76b3 Diff: https://reviews.apache.org/r/29523/diff/ Testing --- Thanks, Manikumar Reddy O
[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269513#comment-14269513 ] Manikumar Reddy commented on KAFKA-1723: Updated reviewboard https://reviews.apache.org/r/29523/diff/ against branch origin/0.8.2 make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch, KAFKA-1723_2015-01-08_22:02:22.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 29714: Patch for KAFKA-1810
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29714/ --- Review request for kafka. Bugs: KAFKA-1810 https://issues.apache.org/jira/browse/KAFKA-1810 Repository: kafka Description --- KAFKA-1810 Diffs - core/src/main/scala/kafka/network/IPFilter.scala PRE-CREATION core/src/main/scala/kafka/network/SocketServer.scala 39b1651b680b2995cedfde95d74c086d9c6219ef core/src/main/scala/kafka/server/KafkaConfig.scala 6e26c5436feb4629d17f199011f3ebb674aa767f core/src/main/scala/kafka/server/KafkaServer.scala 1691ad7fc80ca0b112f68e3ea0cbab265c75b26b core/src/main/scala/kafka/utils/VerifiableProperties.scala 2ffc7f452dc7a1b6a06ca7a509ed49e1ab3d1e68 core/src/test/scala/unit/kafka/network/IPFilterTest.scala PRE-CREATION core/src/test/scala/unit/kafka/network/SocketServerTest.scala 78b431f9c88cca1bc5e430ffd41083d0e92b7e75 core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala 2377abe4933e065d037828a214c3a87e1773a8ef Diff: https://reviews.apache.org/r/29714/diff/ Testing --- Thanks, Jeff Holoman
[jira] [Updated] (KAFKA-1845) KafkaConfig should use ConfigDef
[ https://issues.apache.org/jira/browse/KAFKA-1845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrii Biletskyi updated KAFKA-1845: Issue Type: Sub-task (was: Improvement) Parent: KAFKA-1694 KafkaConfig should use ConfigDef - Key: KAFKA-1845 URL: https://issues.apache.org/jira/browse/KAFKA-1845 Project: Kafka Issue Type: Sub-task Reporter: Gwen Shapira Labels: newbie ConfigDef is already used for the new producer and for TopicConfig. Will be nice to standardize and use one configuration and validation library across the board. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper
Martin Schinz created KAFKA-1847: Summary: Update Readme to reflect changes in gradle wrapper Key: KAFKA-1847 URL: https://issues.apache.org/jira/browse/KAFKA-1847 Project: Kafka Issue Type: Improvement Components: build Affects Versions: 0.8.2, 0.8.3, 0.9.0 Reporter: Martin Schinz Priority: Minor KAFKA-1490 removed a dependency on a binary. This [comment|https://issues.apache.org/jira/browse/KAFKA-1490?focusedCommentId=14157865page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14157865] reflects the changes to the build process. The documentation should be updated to make users aware of this requirement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-1810: Attachment: KAFKA-1810.patch Add IP Filtering / Whitelists-Blacklists - Key: KAFKA-1810 URL: https://issues.apache.org/jira/browse/KAFKA-1810 Project: Kafka Issue Type: New Feature Components: core, network Reporter: Jeff Holoman Assignee: Jeff Holoman Priority: Minor Fix For: 0.8.3 Attachments: KAFKA-1810.patch While longer-term goals of security in Kafka are on the roadmap there exists some value for the ability to restrict connection to Kafka brokers based on IP address. This is not intended as a replacement for security but more of a precaution against misconfiguration and to provide some level of control to Kafka administrators about who is reading/writing to their cluster. 1) In some organizations software administration vs o/s systems administration and network administration is disjointed and not well choreographed. Providing software administrators the ability to configure their platform relatively independently (after initial configuration) from Systems administrators is desirable. 2) Configuration and deployment is sometimes error prone and there are situations when test environments could erroneously read/write to production environments 3) An additional precaution against reading sensitive data is typically welcomed in most large enterprise deployments. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Holoman updated KAFKA-1810: Status: Patch Available (was: Open) Add IP Filtering / Whitelists-Blacklists - Key: KAFKA-1810 URL: https://issues.apache.org/jira/browse/KAFKA-1810 Project: Kafka Issue Type: New Feature Components: core, network Reporter: Jeff Holoman Assignee: Jeff Holoman Priority: Minor Fix For: 0.8.3 Attachments: KAFKA-1810.patch While longer-term goals of security in Kafka are on the roadmap there exists some value for the ability to restrict connection to Kafka brokers based on IP address. This is not intended as a replacement for security but more of a precaution against misconfiguration and to provide some level of control to Kafka administrators about who is reading/writing to their cluster. 1) In some organizations software administration vs o/s systems administration and network administration is disjointed and not well choreographed. Providing software administrators the ability to configure their platform relatively independently (after initial configuration) from Systems administrators is desirable. 2) Configuration and deployment is sometimes error prone and there are situations when test environments could erroneously read/write to production environments 3) An additional precaution against reading sensitive data is typically welcomed in most large enterprise deployments. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269567#comment-14269567 ] Jeff Holoman commented on KAFKA-1810: - Created reviewboard https://reviews.apache.org/r/29714/diff/ against branch origin/trunk Add IP Filtering / Whitelists-Blacklists - Key: KAFKA-1810 URL: https://issues.apache.org/jira/browse/KAFKA-1810 Project: Kafka Issue Type: New Feature Components: core, network Reporter: Jeff Holoman Assignee: Jeff Holoman Priority: Minor Fix For: 0.8.3 Attachments: KAFKA-1810.patch While longer-term goals of security in Kafka are on the roadmap there exists some value for the ability to restrict connection to Kafka brokers based on IP address. This is not intended as a replacement for security but more of a precaution against misconfiguration and to provide some level of control to Kafka administrators about who is reading/writing to their cluster. 1) In some organizations software administration vs o/s systems administration and network administration is disjointed and not well choreographed. Providing software administrators the ability to configure their platform relatively independently (after initial configuration) from Systems administrators is desirable. 2) Configuration and deployment is sometimes error prone and there are situations when test environments could erroneously read/write to production environments 3) An additional precaution against reading sensitive data is typically welcomed in most large enterprise deployments. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1810) Add IP Filtering / Whitelists-Blacklists
[ https://issues.apache.org/jira/browse/KAFKA-1810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269579#comment-14269579 ] Jeff Holoman commented on KAFKA-1810: - This patch is a first pass at implementing IP Filtering logic. It requires defining two additional properties: security.ip.filter.rule.type security.ip.filter.list. The list of IP's are specified in CIDR notation: http://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing. This implementation supports whitelisting (allow config value) or blacklisting (deny), mutually exclusive. The parameters are passed to socket server and validated upon startup.(I'd like to move most of the validation logic per KAFKA-1845). An exception is thrown and the server shutdown in the event of misconfiguration of these parameters. The check against the list is in the Acceptor thread, and if the rule check fails, the socket is closed. There are a lot of tests included in the patch but if there are suggestions for more please let me know. Add IP Filtering / Whitelists-Blacklists - Key: KAFKA-1810 URL: https://issues.apache.org/jira/browse/KAFKA-1810 Project: Kafka Issue Type: New Feature Components: core, network Reporter: Jeff Holoman Assignee: Jeff Holoman Priority: Minor Fix For: 0.8.3 Attachments: KAFKA-1810.patch While longer-term goals of security in Kafka are on the roadmap there exists some value for the ability to restrict connection to Kafka brokers based on IP address. This is not intended as a replacement for security but more of a precaution against misconfiguration and to provide some level of control to Kafka administrators about who is reading/writing to their cluster. 1) In some organizations software administration vs o/s systems administration and network administration is disjointed and not well choreographed. Providing software administrators the ability to configure their platform relatively independently (after initial configuration) from Systems administrators is desirable. 2) Configuration and deployment is sometimes error prone and there are situations when test environments could erroneously read/write to production environments 3) An additional precaution against reading sensitive data is typically welcomed in most large enterprise deployments. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269585#comment-14269585 ] Manikumar Reddy commented on KAFKA-1723: 1. Added MetricName object and related changes. This patch introduces changes to producer interface. 2. Sensor object still has name as unique key . To make name unique, still the code encodes topic name+ name , node name + name into the name string. Because it is internal code, i have not changed. If required, we can use MetricName object for Sensor name also. 3. Currently producer client.id (ProducerConfig.CLIENT_ID_CONFIG) is not mandatory. Can we make this mandatory? or Can we auto-generate this (producer-1,2), if not supplied ? client.id is required to make unique MBean name. if multiple producers (without client.id) are running on same JVM, then we will miss some MBeans. make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch, KAFKA-1723_2015-01-08_22:02:22.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269478#comment-14269478 ] Joe Stein commented on KAFKA-1847: -- Hi [~thinktainer] this is already updated https://github.com/apache/kafka/blob/0.8.2/README.md#first-bootstrap-and-download-the-wrapper what more were you thinking should be done? Update Readme to reflect changes in gradle wrapper -- Key: KAFKA-1847 URL: https://issues.apache.org/jira/browse/KAFKA-1847 Project: Kafka Issue Type: Improvement Components: build Affects Versions: 0.8.2, 0.8.3, 0.9.0 Reporter: Martin Schinz Priority: Minor Labels: documentation, easyfix, newbie KAFKA-1490 removed a dependency on a binary. This [comment|https://issues.apache.org/jira/browse/KAFKA-1490?focusedCommentId=14157865page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14157865] reflects the changes to the build process. The documentation should be updated to make users aware of this requirement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269489#comment-14269489 ] Manikumar Reddy commented on KAFKA-1723: Updated reviewboard https://reviews.apache.org/r/29523/diff/ against branch origin/0.8.2 make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1723: --- Attachment: KAFKA-1723_2015-01-08_21:41:13.patch make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29523: Patch for KAFKA-1723
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:13 p.m.) Review request for kafka. Bugs: KAFKA-1723 https://issues.apache.org/jira/browse/KAFKA-1723 Repository: kafka Description (updated) --- Standard JMX MBean Naming is implemented;Addresing Jay's comments Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 1bce50185273dbdbc131fbc9c7f5f3e9c346517a clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 7f8a41c4bf437711685a8271a4d3c83a176dd957 clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 8cab16c0a0bdb671fea1fc2fc2694247f66cc971 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 3053f2745c8e5f6e3b75826d3749656f150878db clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java aa91e1444a49c55870b9a7a32086fa2b04471fba clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 84a7a07269c51ccc22ebb4ff9797292d07ba778e clients/src/main/java/org/apache/kafka/common/Metric.java b023e8e7c486adf21ed9a554085ab8ad7f3ee038 clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java 29185a6a90d0035d650c7e56ce612a0878cb115c clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 3c312011a7ff7e79c277a89048e7e62ebd6078db clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java a7458b50cb16fbb2b31b857d5b359e65258bbf08 clients/src/main/java/org/apache/kafka/common/metrics/MetricName.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java 49be4019ac03835701c49646920766228ac7ffe9 clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java 25c1faf2887ea02708c1f5b5f822f5299ed86bd6 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java 7365ceb39072a6a1ecf533f5a20830ed1f2cfc72 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java c70d577ada8c099533d4f4ed2e86d37e0a6e6676 clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java fe3c13f319d48b89a4f26b6d78c2c3e31cc50d7e clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java 2f43c49450e1a3d671bd17417dc42941f1858750 clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java 19bea0f1fa1ebf15d86623015ec909b0155e4bd3 clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 clients/src/test/java/org/apache/kafka/test/MetricsBench.java 9d98c1148255455fd801043b59b98fed9d0b76b3 Diff: https://reviews.apache.org/r/29523/diff/ Testing --- Thanks, Manikumar Reddy O
[jira] [Resolved] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Martin Schinz resolved KAFKA-1847. -- Resolution: Invalid Update Readme to reflect changes in gradle wrapper -- Key: KAFKA-1847 URL: https://issues.apache.org/jira/browse/KAFKA-1847 Project: Kafka Issue Type: Improvement Components: build Affects Versions: 0.8.2, 0.8.3, 0.9.0 Reporter: Martin Schinz Priority: Minor Labels: documentation, easyfix, newbie KAFKA-1490 removed a dependency on a binary. This [comment|https://issues.apache.org/jira/browse/KAFKA-1490?focusedCommentId=14157865page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14157865] reflects the changes to the build process. The documentation should be updated to make users aware of this requirement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1847) Update Readme to reflect changes in gradle wrapper
[ https://issues.apache.org/jira/browse/KAFKA-1847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269512#comment-14269512 ] Martin Schinz commented on KAFKA-1847: -- Sorry, I must have missed that when reading initially. Happy to close! Update Readme to reflect changes in gradle wrapper -- Key: KAFKA-1847 URL: https://issues.apache.org/jira/browse/KAFKA-1847 Project: Kafka Issue Type: Improvement Components: build Affects Versions: 0.8.2, 0.8.3, 0.9.0 Reporter: Martin Schinz Priority: Minor Labels: documentation, easyfix, newbie KAFKA-1490 removed a dependency on a binary. This [comment|https://issues.apache.org/jira/browse/KAFKA-1490?focusedCommentId=14157865page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14157865] reflects the changes to the build process. The documentation should be updated to make users aware of this requirement. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector
Aditya Auradkar created KAFKA-1848: -- Summary: Checking shutdown during each iteration of ZookeeperConsumerConnector Key: KAFKA-1848 URL: https://issues.apache.org/jira/browse/KAFKA-1848 Project: Kafka Issue Type: Bug Reporter: Aditya Auradkar Fix For: 0.9.0 In ZookeeperConsumerConnector the syncedRebalance() method checks the isShuttingDown flag before it triggers a rebalance. However, it does not recheck the same value between successive retries which is possible if the consumer is shutting down. This acquires the rebalanceLock and blocks shutdown from completing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: unable to sign up for kafka email distribution
Hi Ben, Your email is received on the mailing list, the kicking-back may be a false alarm. Guozhang On Thu, Jan 8, 2015 at 9:22 AM, Padgett, Ben bpadg...@illumina.com wrote: Unable to confirm myself on kafka email list. Mail is getting kicked back from apache email servers according to our tech department. Thanks! -- -- Guozhang
[jira] [Commented] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector
[ https://issues.apache.org/jira/browse/KAFKA-1848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269841#comment-14269841 ] Aditya Auradkar commented on KAFKA-1848: I've asked Guozhang to assign the ticket to me. Checking shutdown during each iteration of ZookeeperConsumerConnector - Key: KAFKA-1848 URL: https://issues.apache.org/jira/browse/KAFKA-1848 Project: Kafka Issue Type: Bug Reporter: Aditya Auradkar Fix For: 0.9.0 In ZookeeperConsumerConnector the syncedRebalance() method checks the isShuttingDown flag before it triggers a rebalance. However, it does not recheck the same value between successive retries which is possible if the consumer is shutting down. This acquires the rebalanceLock and blocks shutdown from completing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1848) Checking shutdown during each iteration of ZookeeperConsumerConnector
[ https://issues.apache.org/jira/browse/KAFKA-1848?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269866#comment-14269866 ] Guozhang Wang commented on KAFKA-1848: -- Aditya, I have added you to the contributor list. You should be able to assign this to yourself now. Checking shutdown during each iteration of ZookeeperConsumerConnector - Key: KAFKA-1848 URL: https://issues.apache.org/jira/browse/KAFKA-1848 Project: Kafka Issue Type: Bug Reporter: Aditya Auradkar Fix For: 0.9.0 In ZookeeperConsumerConnector the syncedRebalance() method checks the isShuttingDown flag before it triggers a rebalance. However, it does not recheck the same value between successive retries which is possible if the consumer is shutting down. This acquires the rebalanceLock and blocks shutdown from completing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: unable to sign up for kafka email distribution
I haven’t gotten any emails from the email list however. When I joined the Apache Cassandra list I got very frequent emails. Was trying to subscribe at this email dev-subscr...@kafka.apache.orgmailto:%20dev-subscr...@kafka.apache.org. A Thanks! From: Guozhang Wang wangg...@gmail.commailto:wangg...@gmail.com Date: Thursday, January 8, 2015 at 10:40 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org, . . bpadg...@illumina.commailto:bpadg...@illumina.com Subject: Re: unable to sign up for kafka email distribution Hi Ben, Your email is received on the mailing list, the kicking-back may be a false alarm. Guozhang On Thu, Jan 8, 2015 at 9:22 AM, Padgett, Ben bpadg...@illumina.commailto:bpadg...@illumina.com wrote: Unable to confirm myself on kafka email list. Mail is getting kicked back from apache email servers according to our tech department. Thanks! -- -- Guozhang
Re: Review Request 29714: Patch for KAFKA-1810
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29714/ --- (Updated Jan. 8, 2015, 7:14 p.m.) Review request for kafka. Bugs: KAFKA-1810 https://issues.apache.org/jira/browse/KAFKA-1810 Repository: kafka Description --- KAFKA-1810 Diffs - core/src/main/scala/kafka/network/IPFilter.scala PRE-CREATION core/src/main/scala/kafka/network/SocketServer.scala 39b1651b680b2995cedfde95d74c086d9c6219ef core/src/main/scala/kafka/server/KafkaConfig.scala 6e26c5436feb4629d17f199011f3ebb674aa767f core/src/main/scala/kafka/server/KafkaServer.scala 1691ad7fc80ca0b112f68e3ea0cbab265c75b26b core/src/main/scala/kafka/utils/VerifiableProperties.scala 2ffc7f452dc7a1b6a06ca7a509ed49e1ab3d1e68 core/src/test/scala/unit/kafka/network/IPFilterTest.scala PRE-CREATION core/src/test/scala/unit/kafka/network/SocketServerTest.scala 78b431f9c88cca1bc5e430ffd41083d0e92b7e75 core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala 2377abe4933e065d037828a214c3a87e1773a8ef Diff: https://reviews.apache.org/r/29714/diff/ Testing (updated) --- This code centers around a new class, CIDRRange in IPFilter.scala. The IPFilter class is created and holds two fields, the ruleType (allow|deny|none) and a list of CIDRRange objects. This is used in the Socket Server acceptor thread. The check does an exists on the value in the list if the rule type is allow or deny. On object creation, we pre-calculate the lower and upper range values and store those as a BigInt. The overhead of the check should be fairly minimal as it involves converting the incoming IP Address to a BigInt and then just doing a compare to the low/high values. In writing this review up I realized that I can optimize this further to convert to bigint first and move that conversion out of the range check, which I can address. Testing covers the CIDRRange and IPFilter classes and validation of IPV6, IPV4, and configurations. Additionally the functionality is tested in SocketServerTest. Other changes are just to assist in configuration. I modified the SocketServerTest to use a method for grabbing the Socket server to make the code a bit more concise. One key point is that, if there is an error in configuration, we halt the startup of the broker. The thinking there is that if you screw up security-related configs, you want to know about it right away rather than silently accept connections. (thanks Joe Stein for the input). There are two new exceptions realted to this functionality, one to handle configuration errors, and one to handle blocking the request. Currently the level is set to INFO. Does it make sense to move this to WARN ? Thanks, Jeff Holoman
Review Request 29724: Patch for KAFKA-1566
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29724/ --- Review request for kafka. Bugs: KAFKA-1566 https://issues.apache.org/jira/browse/KAFKA-1566 Repository: kafka Description --- KAFKA-1566. Kafka environment configuration (kafka-env.sh). Diffs - bin/kafka-run-class.sh ce3a4d06a27f66a33d729a15207b2d226b100c6a bin/windows/kafka-run-class.bat 9df3d2b45236b4f06d55a89c84afcf0ab9f5d0f2 config/kafka-env.cmd PRE-CREATION config/kafka-env.sh PRE-CREATION Diff: https://reviews.apache.org/r/29724/diff/ Testing --- Thanks, Sriharsha Chintalapani
Re: Review Request 29523: Patch for KAFKA-1723
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/#review67246 --- Thanks for the patch. Looks good overall. A few minor comments below. clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java https://reviews.apache.org/r/29523/#comment111226 Should use lower case param names. clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java https://reviews.apache.org/r/29523/#comment111234 Should this be passed in? Or perhaps we can define this as a constant and reference it here. clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java https://reviews.apache.org/r/29523/#comment98 Should we use message-size-avg as the metrics name? clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java https://reviews.apache.org/r/29523/#comment111211 Param name is no longer valid. This applies to a few other places as well. clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java https://reviews.apache.org/r/29523/#comment111203 Do we need to make this synchronized since the other addMetric is already synchronized. clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java https://reviews.apache.org/r/29523/#comment111212 Param name is no longer valid. This applies to a few other places as well. clients/src/main/java/org/apache/kafka/common/network/Selector.java https://reviews.apache.org/r/29523/#comment111251 It's probably better to just reference a ProducerMetrics constant. clients/src/main/java/org/apache/kafka/common/network/Selector.java https://reviews.apache.org/r/29523/#comment111252 Instead of node-+node, it probably can just be node. - Jun Rao On Jan. 8, 2015, 4:34 p.m., Manikumar Reddy O wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29523/ --- (Updated Jan. 8, 2015, 4:34 p.m.) Review request for kafka. Bugs: KAFKA-1723 https://issues.apache.org/jira/browse/KAFKA-1723 Repository: kafka Description --- Standard JMX MBean Naming is implemented;Addresing Jay's comments Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 1bce50185273dbdbc131fbc9c7f5f3e9c346517a clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 7f8a41c4bf437711685a8271a4d3c83a176dd957 clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 8cab16c0a0bdb671fea1fc2fc2694247f66cc971 clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 3053f2745c8e5f6e3b75826d3749656f150878db clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java aa91e1444a49c55870b9a7a32086fa2b04471fba clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 84a7a07269c51ccc22ebb4ff9797292d07ba778e clients/src/main/java/org/apache/kafka/common/Metric.java b023e8e7c486adf21ed9a554085ab8ad7f3ee038 clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java 29185a6a90d0035d650c7e56ce612a0878cb115c clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java 3c312011a7ff7e79c277a89048e7e62ebd6078db clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java a7458b50cb16fbb2b31b857d5b359e65258bbf08 clients/src/main/java/org/apache/kafka/common/metrics/MetricName.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java 49be4019ac03835701c49646920766228ac7ffe9 clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java 25c1faf2887ea02708c1f5b5f822f5299ed86bd6 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java 7365ceb39072a6a1ecf533f5a20830ed1f2cfc72 clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java c70d577ada8c099533d4f4ed2e86d37e0a6e6676 clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java fe3c13f319d48b89a4f26b6d78c2c3e31cc50d7e clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe
[jira] [Updated] (KAFKA-1566) Kafka environment configuration (kafka-env.sh)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1566: -- Attachment: KAFKA-1566.patch Kafka environment configuration (kafka-env.sh) -- Key: KAFKA-1566 URL: https://issues.apache.org/jira/browse/KAFKA-1566 Project: Kafka Issue Type: Improvement Components: tools Reporter: Cosmin Lehene Assignee: Cosmin Lehene Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1566.patch It would be useful (especially for automated deployments) to have an environment configuration file that could be sourced from the launcher files (e.g. kafka-run-server.sh). This is how this could look like kafka-env.sh {code} export KAFKA_JVM_PERFORMANCE_OPTS=-XX:+UseCompressedOops -XX:+DisableExplicitGC -Djava.awt.headless=true \ -XX:+UseG1GC -XX:PermSize=48m -XX:MaxPermSize=48m -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35' % export KAFKA_HEAP_OPTS='-Xmx1G -Xms1G' % export KAFKA_LOG4J_OPTS=-Dkafka.logs.dir=/var/log/kafka {code} kafka-server-start.sh {code} ... source $base_dir/config/kafka-env.sh ... {code} This approach is consistent with Hadoop and HBase. However the idea here is to be able to set these values in a single place without having to edit startup scripts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1566) Kafka environment configuration (kafka-env.sh)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270026#comment-14270026 ] Sriharsha Chintalapani commented on KAFKA-1566: --- Created reviewboard https://reviews.apache.org/r/29724/diff/ against branch origin/trunk Kafka environment configuration (kafka-env.sh) -- Key: KAFKA-1566 URL: https://issues.apache.org/jira/browse/KAFKA-1566 Project: Kafka Issue Type: Improvement Components: tools Reporter: Cosmin Lehene Assignee: Cosmin Lehene Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1566.patch It would be useful (especially for automated deployments) to have an environment configuration file that could be sourced from the launcher files (e.g. kafka-run-server.sh). This is how this could look like kafka-env.sh {code} export KAFKA_JVM_PERFORMANCE_OPTS=-XX:+UseCompressedOops -XX:+DisableExplicitGC -Djava.awt.headless=true \ -XX:+UseG1GC -XX:PermSize=48m -XX:MaxPermSize=48m -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35' % export KAFKA_HEAP_OPTS='-Xmx1G -Xms1G' % export KAFKA_LOG4J_OPTS=-Dkafka.logs.dir=/var/log/kafka {code} kafka-server-start.sh {code} ... source $base_dir/config/kafka-env.sh ... {code} This approach is consistent with Hadoop and HBase. However the idea here is to be able to set these values in a single place without having to edit startup scripts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1566) Kafka environment configuration (kafka-env.sh)
[ https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1566: -- Assignee: Sriharsha Chintalapani (was: Cosmin Lehene) Status: Patch Available (was: Open) Kafka environment configuration (kafka-env.sh) -- Key: KAFKA-1566 URL: https://issues.apache.org/jira/browse/KAFKA-1566 Project: Kafka Issue Type: Improvement Components: tools Reporter: Cosmin Lehene Assignee: Sriharsha Chintalapani Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1566.patch It would be useful (especially for automated deployments) to have an environment configuration file that could be sourced from the launcher files (e.g. kafka-run-server.sh). This is how this could look like kafka-env.sh {code} export KAFKA_JVM_PERFORMANCE_OPTS=-XX:+UseCompressedOops -XX:+DisableExplicitGC -Djava.awt.headless=true \ -XX:+UseG1GC -XX:PermSize=48m -XX:MaxPermSize=48m -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35' % export KAFKA_HEAP_OPTS='-Xmx1G -Xms1G' % export KAFKA_LOG4J_OPTS=-Dkafka.logs.dir=/var/log/kafka {code} kafka-server-start.sh {code} ... source $base_dir/config/kafka-env.sh ... {code} This approach is consistent with Hadoop and HBase. However the idea here is to be able to set these values in a single place without having to edit startup scripts. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 28769: Patch for KAFKA-1809
On Jan. 7, 2015, 3:01 a.m., Jun Rao wrote: Thanks for the patch. It's a lot work! A few general comments below, in addition to the more detailed comments. 1. Formatting: add a space after comma in function signature and function calls. So instead of def foo(a: A,b: B) use def foo(a: A, b: B) 2. In some of the files, imports can be optimized. 3. Some new files are missing the license header. 4. It seems that the client needs to know the security protocol in addition to the port. Otherwise, it doesn't know which protocol to use to establish the socket connection. So, perhaps the broker.list should now be protocol://host:port? for #4 - we currently have security.protocol parameter for producer and consumer - reason being that: 1. broker.list may specify multiple brokers, but I think we want to keep communication to one protocol per client. We can validate a single protocol in the config, but it is still a bit wierd - asking users to type protocol multiple times when it has to be the same. 2. Zookeeper consumer currently doesn't have broker.list, they have zookeeper list, but it still needs to know the protocol - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review66944 --- On Jan. 6, 2015, 7:46 p.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/ --- (Updated Jan. 6, 2015, 7:46 p.m.) Review request for kafka. Bugs: KAFKA-1809 https://issues.apache.org/jira/browse/KAFKA-1809 Repository: kafka Description --- first commit of refactoring. changed topicmetadata to include brokerendpoints and fixed few unit tests fixing systest and support for binding to default address fixed system tests fix default address binding and ipv6 support fix some issues regarding endpoint parsing. Also, larger segments for systest make the validation much faster added link to security wiki in doc fixing unit test after rename of ProtocolType to SecurityProtocol Following Joe's advice, added security protocol enum on client side, and modified protocol to use ID instead of string. validate producer config against enum add a second protocol for testing and modify SocketServerTests to check on multi-ports Reverted the metadata request changes and removed the explicit security protocol argument. Instead the socketserver will determine the protocol based on the port and add this to the request bump version for UpdateMetadataRequest and added support for rolling upgrades with new config following tests - fixed LeaderAndISR protocol and ZK registration for backward compatibility cleaned up some changes that were actually not necessary. hopefully making this patch slightly easier to review undoing some changes that don't belong here bring back config lost in cleanup fixes neccessary for an all non-plaintext cluster to work Diffs - clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 525b95e98010cd2053eacd8c321d079bcac2f910 clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/utils/Utils.java 527dd0f9c47fce7310b7a37a9b95bf87f1b9c292 clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java 6e37ea553f73d9c584641c48c56dbf6e62ba5f88 clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java a39fab532f73148316a56c0f8e9197f38ea66f79 config/server.properties b0e4496a8ca736b6abe965a430e8ce87b0e8287f core/src/main/scala/kafka/admin/AdminUtils.scala 28b12c7b89a56c113b665fbde1b95f873f8624a3 core/src/main/scala/kafka/admin/TopicCommand.scala 285c0333ff43543d3e46444c1cd9374bb883bb59 core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala 84f60178f6ebae735c8aa3e79ed93fe21ac4aea7 core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala 4ff7e8f8cc695551dd5d2fe65c74f6b6c571e340 core/src/main/scala/kafka/api/TopicMetadata.scala 0190076df0adf906ecd332284f222ff974b315fc core/src/main/scala/kafka/api/TopicMetadataResponse.scala 92ac4e687be22e4800199c0666bfac5e0059e5bb core/src/main/scala/kafka/api/UpdateMetadataRequest.scala 530982e36b17934b8cc5fb668075a5342e142c59 core/src/main/scala/kafka/client/ClientUtils.scala ebba87f0566684c796c26cb76c64b4640a5ccfde core/src/main/scala/kafka/cluster/Broker.scala 0060add008bb3bc4b0092f2173c469fce0120be6 core/src/main/scala/kafka/cluster/BrokerEndPoint.scala PRE-CREATION core/src/main/scala/kafka/cluster/EndPoint.scala PRE-CREATION
[jira] [Commented] (KAFKA-1723) make the metrics name in new producer more standard
[ https://issues.apache.org/jira/browse/KAFKA-1723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270066#comment-14270066 ] Jun Rao commented on KAFKA-1723: Thanks for the patch. 1. Yes, I think this is needed. 2. That's probably fine for now. 3. Yes, I would vote for making client.id mandatory. [~jkreps], your thoughts? make the metrics name in new producer more standard --- Key: KAFKA-1723 URL: https://issues.apache.org/jira/browse/KAFKA-1723 Project: Kafka Issue Type: Sub-task Components: clients Affects Versions: 0.8.2 Reporter: Jun Rao Assignee: Manikumar Reddy Priority: Blocker Fix For: 0.8.2 Attachments: KAFKA-1723.patch, KAFKA-1723_2015-01-08_21:41:13.patch, KAFKA-1723_2015-01-08_22:02:22.patch The jmx name in the new producer looks like the following: kafka.producer.myclientid:type=mytopic However, this can be ambiguous since we allow . in client id and topic. -- This message was sent by Atlassian JIRA (v6.3.4#6332)