[jira] [Resolved] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1938. -- Resolution: Fixed [doc] Quick start example should reference appropriate Kafka version Key: KAFKA-1938 URL: https://issues.apache.org/jira/browse/KAFKA-1938 Project: Kafka Issue Type: Improvement Components: website Affects Versions: 0.8.2.0 Reporter: Stevo Slavic Assignee: Manikumar Reddy Priority: Trivial Attachments: remove-081-references-1.patch, remove-081-references.patch Kafka 0.8.2.0 documentation, quick start example on https://kafka.apache.org/documentation.html#quickstart in step 1 links and instructs reader to download Kafka 0.8.1.1. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318628#comment-14318628 ] Jay Kreps commented on KAFKA-1938: -- Committed. Let me know if you see any issues. [doc] Quick start example should reference appropriate Kafka version Key: KAFKA-1938 URL: https://issues.apache.org/jira/browse/KAFKA-1938 Project: Kafka Issue Type: Improvement Components: website Affects Versions: 0.8.2.0 Reporter: Stevo Slavic Assignee: Manikumar Reddy Priority: Trivial Attachments: remove-081-references-1.patch, remove-081-references.patch Kafka 0.8.2.0 documentation, quick start example on https://kafka.apache.org/documentation.html#quickstart in step 1 links and instructs reader to download Kafka 0.8.1.1. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1949) Update 0.8.3.0 docs with upgrade process
Gwen Shapira created KAFKA-1949: --- Summary: Update 0.8.3.0 docs with upgrade process Key: KAFKA-1949 URL: https://issues.apache.org/jira/browse/KAFKA-1949 Project: Kafka Issue Type: Task Reporter: Gwen Shapira KAFKA-1809 has a protocol bump, which requires an upgrade. The instructions are here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-2+-+Refactor+brokers+to+allow+listening+on+multiple+ports+and+IPs We need to add them to the official documentation. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 30946: Fix KAFKA-1948
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30946/ --- Review request for kafka. Bugs: KAFKA-1948 https://issues.apache.org/jira/browse/KAFKA-1948 Repository: kafka Description --- KAFKA-1948.v1 Diffs - core/src/main/scala/kafka/controller/KafkaController.scala 66df6d2fbdbdd556da6bea0df84f93e0472c8fbf core/src/test/scala/integration/kafka/api/ConsumerTest.scala 798f035df52e405176f558806584ce25e8c392ac Diff: https://reviews.apache.org/r/30946/diff/ Testing --- Thanks, Guozhang Wang
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients as to which endpoint should be selected). I believe I asked about that previously, and I didn't track what the final outcome was or even if it was discussed further. -Todd On Wed, Feb 11, 2015 at 4:38 PM, Gwen Shapira gshap...@cloudera.com wrote: Added Jun's notes to the KIP (Thanks for explaining so clearly, Jun. I was clearly struggling with this...) and removed the reference to use.new.wire.protocol. On Wed, Feb 11, 2015 at 4:19 PM, Joel Koshy jjkosh...@gmail.com wrote: The description that Jun gave for (2) was the detail I was looking for - Gwen can you update the KIP with that for completeness/clarity? I'm +1 as well overall. However, I think it would be good if we also get an ack from someone who is more experienced on the operations side (say, Todd) to review especially the upgrade plan. On Wed, Feb 11, 2015 at 09:40:50AM -0800, Jun Rao wrote: +1 for proposed changes in 1 and 2. 1. The impact is that if someone uses SimpleConsumer and references Broker explicitly, the application needs code change to compile with 0.8.3. Since SimpleConsumer is not widely used, breaking the API in SimpleConsumer but maintaining overall code cleanness seems to be a better tradeoff. 2. For clarification, the issue is the following. In 0.8.3, we will be evolving the wire protocol of UpdateMedataRequest (to send info about endpoints for different security protocols). Since this is used in intra-cluster communication, we need to do the upgrade in two steps. The idea is that in 0.8.3, we will default wire.protocol.version to 0.8.2. When upgrading to 0.8.3, in step 1, we do a rolling upgrade to 0.8.3. After step 1, all brokers will be capable for processing the new protocol in 0.8.3, but without actually using it. In step 2, we configure wire.protocol.version to 0.8.3 in each broker and do another rolling restart. After step 2, all brokers will start using the new protocol in 0.8.3. Let's say that in the next release 0.9, we are changing the intra-cluster wire protocol again. We will do the similar thing: defaulting wire.protocol.version to 0.8.3 in 0.9 so that people can upgrade from 0.8.3 to 0.9 in two steps. For people who want to upgrade from 0.8.2 to 0.9 directly, they will have to configure wire.protocol.version to 0.8.2 first and then do the two-step upgrade to 0.9. Gwen, In KIP2, there is still a reference to use.new.protocol. This needs to be removed. Also, would it be better to use intra.cluster.wire.protocol.version since this only applies to the wire protocol among brokers? Others, The patch in KAFKA-1809
[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318844#comment-14318844 ] Guozhang Wang commented on KAFKA-1948: -- One issue I found is that in ConsumerTest.testPartitionReassignmentCallback, the shutdown coordinator may also be the leader of the test topic partition, and hence consumer cannot shutdown. This issue exist before KAFKA-1333 but was not exposed. I did not see the test hangs on other cases, [~gwenshap] do you want to apply this patch and see if you still see the test hanging, please let me know the specific test case. kafka.api.consumerTests are hanging --- Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug Reporter: Gwen Shapira Assignee: Guozhang Wang Attachments: KAFKA-1948.patch Noticed today that very often when I run the full test suite, it hangs on kafka.api.consumerTest (not always same test though). It doesn't reproduce 100% of the time, but enough to be very annoying. I also saw it happening on trunk after KAFKA-1333: https://builds.apache.org/view/All/job/Kafka-trunk/389/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1938: --- Attachment: lz4-compression.patch forgot to add LZ4 compression to docs. Attaching a new patch. [doc] Quick start example should reference appropriate Kafka version Key: KAFKA-1938 URL: https://issues.apache.org/jira/browse/KAFKA-1938 Project: Kafka Issue Type: Improvement Components: website Affects Versions: 0.8.2.0 Reporter: Stevo Slavic Assignee: Manikumar Reddy Priority: Trivial Attachments: lz4-compression.patch, remove-081-references-1.patch, remove-081-references.patch Kafka 0.8.2.0 documentation, quick start example on https://kafka.apache.org/documentation.html#quickstart in step 1 links and instructs reader to download Kafka 0.8.1.1. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSS] KIP-8 Add a flush method to the new Java producer
Yes that is a counter-example. I'm okay either way on whether we should have just flush() or have a timeout. Bhavesh, does Jay's explanation a few replies prior address your concern? If so, shall we consider this closed? On Tue, Feb 10, 2015 at 01:36:23PM -0800, Jay Kreps wrote: Yeah we could do that, I guess I just feel like it adds confusion because then you have to think about which timeout you want, when likely you don't want a timeout at all. I guess the pattern I was thinking of was fflush or the java equivalent, which don't have timeouts: http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html#flush() -Jay On Tue, Feb 10, 2015 at 10:41 AM, Joel Koshy jjkosh...@gmail.com wrote: I think tryFlush with a timeout sounds good to me. This is really more for consistency than anything else. I cannot think of any standard blocking calls off the top of my head that don't have a timed variant. E.g., Thread.join, Object.wait, Future.get Either that, or they provide an entirely non-blocking mode (e.g., socketChannel.connect followed by finishConnect) Thanks, Joel On Tue, Feb 10, 2015 at 11:30:47AM -0500, Joe Stein wrote: Jay, The .flush() call seems like it would be the best way if you wanted to-do a clean shutdown of the new producer? So, you could in your code stop all incoming requests producer.flush() system.exit(value) and know pretty much you won't drop anything on the floor. This can be done with the callbacks and futures (sure) but .flush() seems to be the right time to block and a few lines of code, no? ~ Joestein On Tue, Feb 10, 2015 at 11:25 AM, Jay Kreps jay.kr...@gmail.com wrote: Hey Bhavesh, If a broker is not available a new one should be elected to take over, so although the flush might take longer it should still be quick. Even if not this should result in an error not a hang. The cases you enumerated are all covered already--if the user wants to retry that is covered by the retry setting in the client, for all the errors that is considered completion of the request. The post condition of flush isn't that all sends complete successfully, just that they complete. So if you try to send a message that is too big, when flush returns calling .get() on the future should not block and should produce the error. Basically the argument I am making is that the only reason you want to call flush() is to guarantee all the sends complete so if it doesn't guarantee that it will be somewhat confusing. This does mean blocking, but if you don't want to block on the send then you wouldn't call flush(). This has no impact on the block.on.buffer full setting. That impacts what happens when send() can't append to the buffer because it is full. flush() means any message previously sent (i.e. for which send() call has returned) needs to have its request completed. Hope that makes sense. -Jay On Mon, Feb 9, 2015 at 11:52 PM, Bhavesh Mistry mistry.p.bhav...@gmail.com wrote: HI Jay, Imagine, if you have flaky network connection to brokers, and if flush() will be blocked if one of broker is not available ( basically How would be address failure mode and io thread not able to drain records or busy due to pending request. Do you flush() method is only to flush to in mem queue or flush to broker over the network(). Timeout helps with and pushing caller to handle what to do ? e.g re-enqueue records, drop entire batch or one of message is too big cross the limit of max.message.size etc... Also, according to java doc for API The method will block until all previously sent records have completed sending (either successfully or with an error), does this by-pass rule set by for block.on.buffer.full or batch.size when under load. That was my intention, and I am sorry I mixed-up close() method here without knowing that this is only for bulk send. Thanks, Bhavesh On Mon, Feb 9, 2015 at 8:17 PM, Jay Kreps jay.kr...@gmail.com wrote: Yeah I second the problem Guozhang flags with giving flush a timeout. In general failover in Kafka is a bounded thing unless you have brought your Kafka cluster down entirely so I think depending on that bound implicitly is okay. It is possible to make flush() be instead boolean tryFlush(long timeout, TimeUnit unit); But I am somewhat skeptical that people will use this correctly. I.e consider the mirror maker code snippet I gave above, how would one actually recover in this case other than retrying (which the client already does automatically)? After all if you are okay losing data then you don't
[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318838#comment-14318838 ] Guozhang Wang commented on KAFKA-1948: -- Created reviewboard https://reviews.apache.org/r/30946/diff/ against branch origin/trunk kafka.api.consumerTests are hanging --- Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug Reporter: Gwen Shapira Attachments: KAFKA-1948.patch Noticed today that very often when I run the full test suite, it hangs on kafka.api.consumerTest (not always same test though). It doesn't reproduce 100% of the time, but enough to be very annoying. I also saw it happening on trunk after KAFKA-1333: https://builds.apache.org/view/All/job/Kafka-trunk/389/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1948) kafka.api.consumerTests are hanging
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1948: - Attachment: KAFKA-1948.patch kafka.api.consumerTests are hanging --- Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug Reporter: Gwen Shapira Attachments: KAFKA-1948.patch Noticed today that very often when I run the full test suite, it hangs on kafka.api.consumerTest (not always same test though). It doesn't reproduce 100% of the time, but enough to be very annoying. I also saw it happening on trunk after KAFKA-1333: https://builds.apache.org/view/All/job/Kafka-trunk/389/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 28769: Patch for KAFKA-1809
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review72226 --- clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java https://reviews.apache.org/r/28769/#comment118255 It seems that we rely on the lexicographical ordering of the enum name for ordering. This may cause problems when we have versions like 0.10.0.0. So, we probably need a more reliable way for ordering. - Jun Rao On Feb. 3, 2015, 6:52 p.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/ --- (Updated Feb. 3, 2015, 6:52 p.m.) Review request for kafka. Bugs: KAFKA-1809 https://issues.apache.org/jira/browse/KAFKA-1809 Repository: kafka Description --- 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 minor modifications following comments by Jun added missing license formatting clean up imports cleaned up V2 to not include host+port field. Using use.new.protocol flag to decide which version to serialize change endpoints collection in Broker to Map[protocol, endpoint], mostly to be clear that we intend to have one endpoint per protocol validate that listeners and advertised listeners have unique ports and protocols support legacy configs some fixes following rebase Reverted to backward compatible zk registration, changed use.new.protocol to support multiple versions and few minor fixes fixing some issues after rebase modified inter.broker.protocol config to start with security per feedback rebasing on trunk again Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 9a43d668376295f47cea0b8eb26b15a6c73bb39c clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java 0186783e1abd0438d6720d035ee1903b51691e09 clients/src/main/java/org/apache/kafka/common/utils/Utils.java 8a305b0fb46563e9e683c327d9e18de532f291de clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java 4c2ea34815b63174732d58b699e1a0a9e6ec3b6f config/server.properties 1614260b71a658b405bb24157c8f12b1f1031aa5 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 24aaf954dc42e2084454fa5fc9e8f388ea95c756 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
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
Hi, 1. We have another issue with reserving a port for inter-broker communication (at least by not advertising the endpoint). Currently brokers are pretty much normal consumers when replicating. Not exactly, but close. Which mean they also learn about other brokers from the registration in ZK. If a broker fails to advertise an endpoint, no one will know about it - neither clients nor other brokers. If Todd has a good use-case for reserving ports for inter-broker messages (firewalls?), we can support it as a follow-up JIRA. 2. I updated the upgrade guide in the KIP and created KAFKA-1949 to follow up in the docs. Gwen On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients as to which endpoint should be selected). I believe I asked about that previously, and I didn't track what the final outcome was or even if it was discussed further. -Todd On Wed, Feb 11, 2015 at 4:38 PM, Gwen Shapira gshap...@cloudera.com wrote: Added Jun's notes to the KIP (Thanks for explaining so clearly, Jun. I was clearly struggling with this...) and removed the reference to use.new.wire.protocol. On Wed, Feb 11, 2015 at 4:19 PM, Joel Koshy jjkosh...@gmail.com wrote: The description that Jun gave for (2) was the detail I was looking for - Gwen can you update the KIP with that for completeness/clarity? I'm +1 as well overall. However, I think it would be good if we also get an ack from someone who is more experienced on the operations side (say, Todd) to review especially the upgrade plan. On Wed, Feb 11, 2015 at 09:40:50AM -0800, Jun Rao wrote: +1 for proposed changes in 1 and 2. 1. The impact is that if someone uses SimpleConsumer and references Broker explicitly, the application needs code change to compile with 0.8.3. Since SimpleConsumer is not widely used, breaking the API in SimpleConsumer but maintaining overall code cleanness seems to be a better tradeoff. 2. For clarification, the issue is the following. In 0.8.3, we will be evolving the wire protocol of UpdateMedataRequest (to send info about endpoints for different security protocols). Since this is used in intra-cluster communication, we need to do the upgrade in two steps. The idea is that in 0.8.3, we will default wire.protocol.version to 0.8.2. When upgrading to 0.8.3, in step 1, we do a rolling upgrade to 0.8.3. After step 1, all brokers will be capable for processing the new protocol in 0.8.3, but without actually using it. In step 2, we configure wire.protocol.version to 0.8.3 in each broker and do another rolling
Review Request 30919: Patch for KAFKA-1947
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30919/ --- Review request for kafka. Bugs: KAFKA-1947 https://issues.apache.org/jira/browse/KAFKA-1947 Repository: kafka Description --- Fix https://issues.apache.org/jira/browse/KAFKA-1947 Diffs - core/src/main/scala/kafka/admin/TopicCommand.scala 285c0333ff43543d3e46444c1cd9374bb883bb59 Diff: https://reviews.apache.org/r/30919/diff/ Testing --- Thanks, Honghai Chen
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1711) WARN Property topic is not valid when running console producer
[ https://issues.apache.org/jira/browse/KAFKA-1711?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318989#comment-14318989 ] Gwen Shapira commented on KAFKA-1711: - Since KAFKA-1824 is open, I don't think we need both? WARN Property topic is not valid when running console producer -- Key: KAFKA-1711 URL: https://issues.apache.org/jira/browse/KAFKA-1711 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2.0 Reporter: Jun Rao Assignee: Joe Crobak Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1711.patch bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test [2014-10-17 09:54:23,984] WARN Property topic is not valid (kafka.utils.VerifiableProperties) It would be good if we can get rid of the warning. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
The idea is more about isolating the intra-cluster traffic from the normal clients as much as possible. There's a couple situations we've seen where this would be useful that I can think of immediately: 1) Normal operation - just having the intra-cluster traffic on a separate network interface would allow it to not get overwhelmed by something like a bootstrapping client who is saturating the network interface. We see this fairly often, where the replication falls behind because of heavy traffic from one application. We can always adjust the network threads, but segregating the traffic is the first step. 2) Isolation in case of an error - We have had situations, more than once, where we are needing to rebuild a cluster after a catastrophic problem and the clients are causing that process to take too long, or are causing additional failures. This has mostly come into play with file descriptor limits in the past, but it's certainly not the only situation. Constantly reconnecting clients continue to cause the brokers to fall over while we are trying to recover a down cluster. The only solution was to firewall off all the clients temporarily. This is a great deal more complicated if the brokers and the clients are all operating over the same port. Now, that said, quotas can be a partial solution to this. I don't want to jump the gun on that discussion (because it's going to come up separately and in more detail), but it is possible to structure quotas in a way that will allow the intra-cluster replication to continue to function in the case of high load. That would partially address case 1, but it does nothing for case 2. Additionally, I think it is also desirable to segregate the traffic even with quotas, so that regardless of the client load, the cluster itself is able to be healthy. -Todd On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients as to which endpoint should be selected). I believe I asked about that previously, and I didn't track what the final outcome was or even if it was discussed further. -Todd On Wed, Feb 11, 2015 at 4:38 PM, Gwen Shapira gshap...@cloudera.com wrote: Added Jun's notes to the KIP (Thanks for explaining so clearly, Jun. I was clearly struggling with this...) and removed the reference to use.new.wire.protocol. On Wed, Feb 11, 2015 at 4:19 PM, Joel Koshy jjkosh...@gmail.com wrote: The description that Jun gave for (2) was the detail I was looking for - Gwen can you update the KIP with that for completeness/clarity? I'm +1 as well overall. However, I think it would be good if we also get an ack from someone who is more experienced on the operations side
[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318939#comment-14318939 ] Gwen Shapira commented on KAFKA-1948: - Applied the patch and the tests are passing! Thank you for the quick solution. Non-binding +1 from me, and I hope this will be committed ASAP. kafka.api.consumerTests are hanging --- Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug Reporter: Gwen Shapira Assignee: Guozhang Wang Attachments: KAFKA-1948.patch Noticed today that very often when I run the full test suite, it hangs on kafka.api.consumerTest (not always same test though). It doesn't reproduce 100% of the time, but enough to be very annoying. I also saw it happening on trunk after KAFKA-1333: https://builds.apache.org/view/All/job/Kafka-trunk/389/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
For rolling downgrading of the intra-cluster protocol, we mostly just need to reverse the steps. If none of the brokers have set wire.protocol.version to the newer version, downgrading can be done by just downgrading the jar, followed by a rolling restart. Otherwise, downgrading needs to be done in two steps: (1) Reconfigure wire.protocol.version to an old version and do a rolling restart of the brokers. After this step, all brokers will be using the old protocol for intra-cluster communication. (2) Downgrade the jar and do another round of rolling restart. Gwen, Could you document that in the WIP-2 wiki too? Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients as to which endpoint should be selected). I believe I asked about that previously, and I didn't track what the final outcome was or even if it was discussed further. -Todd On Wed, Feb 11, 2015 at 4:38 PM, Gwen Shapira gshap...@cloudera.com wrote: Added Jun's notes to the KIP (Thanks for explaining so clearly, Jun. I was clearly struggling with this...) and removed the reference to use.new.wire.protocol. On Wed, Feb 11, 2015 at 4:19 PM, Joel Koshy jjkosh...@gmail.com wrote: The description that Jun gave for (2) was the detail I was looking for - Gwen can you update the KIP with that for completeness/clarity? I'm +1 as well overall. However, I think it would be good if we also get an ack from someone who is more experienced on the operations side (say, Todd) to review especially the upgrade plan. On Wed, Feb 11, 2015 at 09:40:50AM -0800, Jun Rao wrote: +1 for proposed changes in 1 and 2. 1. The impact is that if someone uses SimpleConsumer and references Broker explicitly, the application needs code change to compile with 0.8.3. Since SimpleConsumer is not widely used, breaking the API in SimpleConsumer but maintaining overall code cleanness seems to be a better tradeoff. 2. For clarification, the issue is the following. In 0.8.3, we will be evolving the wire protocol of UpdateMedataRequest (to send info about endpoints for different security protocols). Since this is used in intra-cluster communication, we need to do the upgrade in two steps. The idea is that in 0.8.3, we will default wire.protocol.version to 0.8.2. When upgrading to 0.8.3, in step 1, we do a rolling upgrade to 0.8.3. After step 1, all brokers will be capable for processing the new protocol in 0.8.3, but without actually using it. In step 2, we configure wire.protocol.version to 0.8.3 in each broker and do another rolling restart. After step 2, all brokers will start using the new protocol in 0.8.3. Let's say that in the next release 0.9, we are changing the intra-cluster wire protocol again. We will do the similar thing: defaulting wire.protocol.version to 0.8.3 in 0.9 so that people can upgrade from 0.8.3 to 0.9 in two steps. For people who want to upgrade from 0.8.2 to 0.9 directly, they will have to configure wire.protocol.version to 0.8.2 first and then do the two-step upgrade to 0.9. Gwen, In KIP2, there is still a reference to use.new.protocol. This needs to be removed. Also, would it be better to use intra.cluster.wire.protocol.version since this only applies to the wire protocol among brokers? Others, The patch in KAFKA-1809 is almost ready. It would be good to wrap up the discussion on KIP2 soon. So, if you haven't looked at this KIP, please take a look and send your comments. Thanks, Jun On Mon, Jan 26, 2015 at 8:02 PM, Gwen Shapira gshap...@cloudera.com
Re: gradle testAll stuck
Test takes about 8 minutes on my machine. I think testAll tests all 3 Scala versions, so it should be around 25 minutes. Why not use just test, most of the time? On Thu, Feb 12, 2015 at 7:01 AM, Tong Li liton...@us.ibm.com wrote: Manikumar, yes, exactly the same issue. Commented on https://issues.apache.org/jira/browse/KAFKA-1948. Before this problem happens, the testAll normally still takes about half hour? Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com [image: Inactive hide details for Manikumar Reddy ---02/12/2015 09:10:58 AM---This may be due to the recent issue reported by Gwen. htt]Manikumar Reddy ---02/12/2015 09:10:58 AM---This may be due to the recent issue reported by Gwen. https://issues.apache.org/jira/browse/KAFKA-19 From: Manikumar Reddy ku...@nmsworks.co.in To: dev@kafka.apache.org Date: 02/12/2015 09:10 AM Subject: Re: gradle testAll stuck Sent by: manikumar.re...@gmail.com -- This may be due to the recent issue reported by Gwen. https://issues.apache.org/jira/browse/KAFKA-1948 On 2/12/15, Joel Koshy jjkosh...@gmail.com wrote: - Can you enable test logging (see the README) and see if you can figure out which test is getting stuck or taking forever? - A thread-dump may help. On Thu, Feb 12, 2015 at 08:57:11AM -0500, Tong Li wrote: Hi, folks, How are you all doing? New bee here. Run gradle --daemon testAll on 0.8.2 worked and finished in about 30 minutes. pulled down trunk and run samething, always stuck. left it run for overnight, checked in the morning still stuck. even on 0.8.2, it still takes over 30 minutes on my modest dev system. Wonder how you all run gradle and is there any specific settings needed? I am running it on Ubuntu 14.04. Any help or pointer is really appreciated. Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com
Re: Review Request 29912: Patch for KAFKA-1852
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29912/ --- (Updated Feb. 13, 2015, 12:46 a.m.) Review request for kafka. Bugs: KAFKA-1852 https://issues.apache.org/jira/browse/KAFKA-1852 Repository: kafka Description (updated) --- KAFKA-1852. OffsetCommitRequest can commit offset on unknown topic. Added contains method to MetadataCache. Diffs (updated) - core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/MetadataCache.scala 4c70aa7e0157b85de5e24736ebf487239c4571d0 core/src/main/scala/kafka/server/OffsetManager.scala 83d52643028c5628057dc0aa29819becfda61fdb core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 5b93239cdc26b5be7696f4e7863adb9fbe5f0ed5 Diff: https://reviews.apache.org/r/29912/diff/ Testing --- Thanks, Sriharsha Chintalapani
[jira] [Commented] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14319327#comment-14319327 ] Sriharsha Chintalapani commented on KAFKA-1852: --- Updated reviewboard https://reviews.apache.org/r/29912/diff/ against branch origin/trunk 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 Assignee: Sriharsha Chintalapani Attachments: KAFKA-1852.patch, KAFKA-1852_2015-01-19_10:44:01.patch, KAFKA-1852_2015-02-12_16:46:10.patch 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-1852) OffsetCommitRequest can commit offset on unknown topic
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1852: -- Attachment: KAFKA-1852_2015-02-12_16:46:10.patch 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 Assignee: Sriharsha Chintalapani Attachments: KAFKA-1852.patch, KAFKA-1852_2015-01-19_10:44:01.patch, KAFKA-1852_2015-02-12_16:46:10.patch 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] [Assigned] (KAFKA-1796) Sanity check partition command line tools
[ https://issues.apache.org/jira/browse/KAFKA-1796?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat reassigned KAFKA-1796: -- Assignee: Mayuresh Gharat Sanity check partition command line tools - Key: KAFKA-1796 URL: https://issues.apache.org/jira/browse/KAFKA-1796 Project: Kafka Issue Type: Bug Reporter: Guozhang Wang Assignee: Mayuresh Gharat Labels: newbie Fix For: 0.8.3 We need to sanity check the input json has the valid values before triggering the admin process. For example, we have seen a scenario where the json input for partition reassignment tools have partition replica info as {broker-1, broker-1, broker-2} and it is still accepted in ZK and eventually lead to under replicated count, etc. This is partially because we use a Map rather than a Set reading the json input for this case; but in general we need to make sure the input parameters like Json needs to be valid before writing it to ZK. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[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=14319273#comment-14319273 ] Jiangjie Qin commented on KAFKA-1797: - I just realized that we removed the default value of key.serializer and value.serializer in ProducerConfig. This will require the users to provide a long class name of org.apache.kafka.common.serialization.ByterraySerializer in the property file. Just wondering the reason why we removed the default value, because it looks that the users are not likely to know the class name if they just begin with Kafka. 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.0 Reporter: Jun Rao Assignee: Jun Rao Fix For: 0.8.2.0 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] [Comment Edited] (KAFKA-1796) Sanity check partition command line tools
[ https://issues.apache.org/jira/browse/KAFKA-1796?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14319346#comment-14319346 ] Mayuresh Gharat edited comment on KAFKA-1796 at 2/13/15 1:07 AM: - [~nehanarkhede], I went over KAFKA-1678. It did not find any mention of this issue. May be I missed it. Should this be included in that jira? If no, I can give a shot at this one. was (Author: mgharat): [~nehanarkhede], I went over KAFKA-1678. I did not find any mention of this issue. May be I missed it. Should this be included in that jira? If no, I can give a shot at this one. Sanity check partition command line tools - Key: KAFKA-1796 URL: https://issues.apache.org/jira/browse/KAFKA-1796 Project: Kafka Issue Type: Bug Reporter: Guozhang Wang Assignee: Mayuresh Gharat Labels: newbie Fix For: 0.8.3 We need to sanity check the input json has the valid values before triggering the admin process. For example, we have seen a scenario where the json input for partition reassignment tools have partition replica info as {broker-1, broker-1, broker-2} and it is still accepted in ZK and eventually lead to under replicated count, etc. This is partially because we use a Map rather than a Set reading the json input for this case; but in general we need to make sure the input parameters like Json needs to be valid before writing it to ZK. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1852) OffsetCommitRequest can commit offset on unknown topic
[ https://issues.apache.org/jira/browse/KAFKA-1852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14319332#comment-14319332 ] Sriharsha Chintalapani commented on KAFKA-1852: --- Thanks [~jjkoshy] for the review. I added your suggestion please take a look when you get a chance. 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 Assignee: Sriharsha Chintalapani Attachments: KAFKA-1852.patch, KAFKA-1852_2015-01-19_10:44:01.patch, KAFKA-1852_2015-02-12_16:46:10.patch 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)
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
I REALLY like the idea of supporting separate network for inter-broker communication (and probably Zookeeper too). I think its actually a pretty typical configuration in clusters, so I'm surprised we didn't think of it before :) Servers arrive with multiple cards specifically for admin nic vs. clients nic vs storage nic. That said, I'd like to handle it in a separate patch. First because KAFKA-1809 is big enough already, and second because this really deserve its own requirement-gathering and design. Does that make sense? Gwen On Thu, Feb 12, 2015 at 12:34 PM, Todd Palino tpal...@gmail.com wrote: The idea is more about isolating the intra-cluster traffic from the normal clients as much as possible. There's a couple situations we've seen where this would be useful that I can think of immediately: 1) Normal operation - just having the intra-cluster traffic on a separate network interface would allow it to not get overwhelmed by something like a bootstrapping client who is saturating the network interface. We see this fairly often, where the replication falls behind because of heavy traffic from one application. We can always adjust the network threads, but segregating the traffic is the first step. 2) Isolation in case of an error - We have had situations, more than once, where we are needing to rebuild a cluster after a catastrophic problem and the clients are causing that process to take too long, or are causing additional failures. This has mostly come into play with file descriptor limits in the past, but it's certainly not the only situation. Constantly reconnecting clients continue to cause the brokers to fall over while we are trying to recover a down cluster. The only solution was to firewall off all the clients temporarily. This is a great deal more complicated if the brokers and the clients are all operating over the same port. Now, that said, quotas can be a partial solution to this. I don't want to jump the gun on that discussion (because it's going to come up separately and in more detail), but it is possible to structure quotas in a way that will allow the intra-cluster replication to continue to function in the case of high load. That would partially address case 1, but it does nothing for case 2. Additionally, I think it is also desirable to segregate the traffic even with quotas, so that regardless of the client load, the cluster itself is able to be healthy. -Todd On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients as to which endpoint should be selected). I believe I asked about that previously, and I didn't track what the
Re: Review Request 30946: Fix KAFKA-1948
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30946/#review72278 --- Ship it! Ship It! - Jay Kreps On Feb. 12, 2015, 7:36 p.m., Guozhang Wang wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30946/ --- (Updated Feb. 12, 2015, 7:36 p.m.) Review request for kafka. Bugs: KAFKA-1948 https://issues.apache.org/jira/browse/KAFKA-1948 Repository: kafka Description --- KAFKA-1948.v1 Diffs - core/src/main/scala/kafka/controller/KafkaController.scala 66df6d2fbdbdd556da6bea0df84f93e0472c8fbf core/src/test/scala/integration/kafka/api/ConsumerTest.scala 798f035df52e405176f558806584ce25e8c392ac Diff: https://reviews.apache.org/r/30946/diff/ Testing --- Thanks, Guozhang Wang
[jira] [Commented] (KAFKA-1796) Sanity check partition command line tools
[ https://issues.apache.org/jira/browse/KAFKA-1796?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14319346#comment-14319346 ] Mayuresh Gharat commented on KAFKA-1796: [~nehanarkhede], I went over KAFKA-1678. I did not find any mention of this issue. May be I missed it. Should this be included in that jira? If no, I can give a shot at this one. Sanity check partition command line tools - Key: KAFKA-1796 URL: https://issues.apache.org/jira/browse/KAFKA-1796 Project: Kafka Issue Type: Bug Reporter: Guozhang Wang Assignee: Mayuresh Gharat Labels: newbie Fix For: 0.8.3 We need to sanity check the input json has the valid values before triggering the admin process. For example, we have seen a scenario where the json input for partition reassignment tools have partition replica info as {broker-1, broker-1, broker-2} and it is still accepted in ZK and eventually lead to under replicated count, etc. This is partially because we use a Map rather than a Set reading the json input for this case; but in general we need to make sure the input parameters like Json needs to be valid before writing it to ZK. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29647: Patch for KAFKA-1697
On Feb. 12, 2015, 2:31 p.m., Joel Koshy wrote: core/src/main/scala/kafka/server/ReplicaManager.scala, line 271 https://reviews.apache.org/r/29647/diff/6/?file=861625#file861625line271 This is good, but maybe call this canRespondNow ? Since before replication won't apply if there are no errors in the partition appends or if there is no data. Joel Koshy wrote: In fact flipping the condition and calling the helper function delayedRequestRequired may be even clearer good point. Fixing this. On Feb. 12, 2015, 2:31 p.m., Joel Koshy wrote: core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala, line 62 https://reviews.apache.org/r/29647/diff/6/?file=861626#file861626line62 Is this change necessary? Nope, but I need to change the test to use produceRequest from TestUtils and not the API test. It snack in when I moved the test code around. Let me fix this. - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72156 --- On Feb. 12, 2015, 7:14 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 7:14 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317808#comment-14317808 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317803#comment-14317803 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1760) Implement new consumer client
[ https://issues.apache.org/jira/browse/KAFKA-1760?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317786#comment-14317786 ] Hongliang Su commented on KAFKA-1760: - [~jkreps] do you have any docs about how to use the new consumer client? Thank you. Implement new consumer client - Key: KAFKA-1760 URL: https://issues.apache.org/jira/browse/KAFKA-1760 Project: Kafka Issue Type: Sub-task Components: consumer Reporter: Jay Kreps Assignee: Jay Kreps Fix For: 0.8.3 Attachments: KAFKA-1760.patch, KAFKA-1760_2015-01-11_16:57:15.patch, KAFKA-1760_2015-01-18_19:10:13.patch, KAFKA-1760_2015-01-21_08:42:20.patch, KAFKA-1760_2015-01-22_10:03:26.patch, KAFKA-1760_2015-01-22_20:21:56.patch, KAFKA-1760_2015-01-23_13:13:00.patch, KAFKA-1760_2015-01-29_03:20:20.patch Implement a consumer client. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317781#comment-14317781 ] Honghai Chen commented on KAFKA-1947: - The fix is quite directly: When has replicaAssignment, the command should only be createTopic or alterTopic When the command is create topic, and replicaAssignment appear, there should no partitions and replica number. The command told me board is ok, but actually no. But patch uploaded. D:\C\Kafka_Updatekafka-patch-review_.py -b origin/trunk -j KAFKA-1947 Configuring reviewboard url to https://reviews.apache.org Updating your remote branches to pull the latest changes Creating diff against origin/trunk and uploading patch to JIRA KAFKA-1947 Created a new reviewboard D:\C\Kafka_Update can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317802#comment-14317802 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: (was: KAFKA-1947.patch) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317832#comment-14317832 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317775#comment-14317775 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Comment: was deleted (was: Created reviewboard against branch origin/trunk) can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317822#comment-14317822 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317821#comment-14317821 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317835#comment-14317835 ] Honghai Chen edited comment on KAFKA-1947 at 2/12/15 8:57 AM: -- Submit review ... https://reviews.apache.org/r/30919/diff/ Please help check was (Author: waldenchen): Submit review ... https://reviews.apache.org/r/30919/ https://reviews.apache.org/r/30919/diff/ can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317835#comment-14317835 ] Honghai Chen commented on KAFKA-1947: - Submit review ... https://reviews.apache.org/r/30919/ https://reviews.apache.org/r/30919/diff/ can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Jenkins build is back to normal : KafkaPreCommit #4
See https://builds.apache.org/job/KafkaPreCommit/4/changes
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
+1 on separating the end points for different purposes. On 2/12/15, 5:47 PM, Gwen Shapira gshap...@cloudera.com wrote: I REALLY like the idea of supporting separate network for inter-broker communication (and probably Zookeeper too). I think its actually a pretty typical configuration in clusters, so I'm surprised we didn't think of it before :) Servers arrive with multiple cards specifically for admin nic vs. clients nic vs storage nic. That said, I'd like to handle it in a separate patch. First because KAFKA-1809 is big enough already, and second because this really deserve its own requirement-gathering and design. Does that make sense? Gwen On Thu, Feb 12, 2015 at 12:34 PM, Todd Palino tpal...@gmail.com wrote: The idea is more about isolating the intra-cluster traffic from the normal clients as much as possible. There's a couple situations we've seen where this would be useful that I can think of immediately: 1) Normal operation - just having the intra-cluster traffic on a separate network interface would allow it to not get overwhelmed by something like a bootstrapping client who is saturating the network interface. We see this fairly often, where the replication falls behind because of heavy traffic from one application. We can always adjust the network threads, but segregating the traffic is the first step. 2) Isolation in case of an error - We have had situations, more than once, where we are needing to rebuild a cluster after a catastrophic problem and the clients are causing that process to take too long, or are causing additional failures. This has mostly come into play with file descriptor limits in the past, but it's certainly not the only situation. Constantly reconnecting clients continue to cause the brokers to fall over while we are trying to recover a down cluster. The only solution was to firewall off all the clients temporarily. This is a great deal more complicated if the brokers and the clients are all operating over the same port. Now, that said, quotas can be a partial solution to this. I don't want to jump the gun on that discussion (because it's going to come up separately and in more detail), but it is possible to structure quotas in a way that will allow the intra-cluster replication to continue to function in the case of high load. That would partially address case 1, but it does nothing for case 2. Additionally, I think it is also desirable to segregate the traffic even with quotas, so that regardless of the client load, the cluster itself is able to be healthy. -Todd On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is blocked by a firewall rule or other authentication method. This could also allow you to duplicate a security protocol type but segregate it on a different port or interface (if it is unadvertised, there is no ambiguity to the clients
[jira] [Updated] (KAFKA-1913) App hungs when calls producer.send to wrong IP of Kafka broker
[ https://issues.apache.org/jira/browse/KAFKA-1913?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1913: Fix Version/s: (was: 0.8.1.2) 0.8.3 App hungs when calls producer.send to wrong IP of Kafka broker -- Key: KAFKA-1913 URL: https://issues.apache.org/jira/browse/KAFKA-1913 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.8.1.1 Environment: OS X 10.10.1, Java 7, AWS Linux Reporter: Igor Khomenko Assignee: Jun Rao Fix For: 0.8.3 I have next test code to check the Kafka functionality: {code} package com.company; import kafka.common.FailedToSendMessageException; import kafka.javaapi.producer.Producer; import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; import java.util.Date; import java.util.Properties; public class Main { public static void main(String[] args) { Properties props = new Properties(); props.put(metadata.broker.list, 192.168.9.3:9092); props.put(serializer.class, com.company.KafkaMessageSerializer); props.put(request.required.acks, 1); ProducerConfig config = new ProducerConfig(props); // The first is the type of the Partition key, the second the type of the message. ProducerString, String messagesProducer = new ProducerString, String(config); // Send String topicName = my_messages; String message = hello world; KeyedMessageString, String data = new KeyedMessageString, String(topicName, message); try { System.out.println(new Date() + : sending...); messagesProducer.send(data); System.out.println(new Date() + : sent); }catch (FailedToSendMessageException e){ System.out.println(e: + e); e.printStackTrace(); }catch (Exception exc){ System.out.println(e: + exc); exc.printStackTrace(); } } } {code} {code} package com.company; import kafka.serializer.Encoder; import kafka.utils.VerifiableProperties; /** * Created by igorkhomenko on 2/2/15. */ public class KafkaMessageSerializer implements EncoderString { public KafkaMessageSerializer(VerifiableProperties verifiableProperties) { /* This constructor must be present for successful compile. */ } @Override public byte[] toBytes(String entity) { byte [] serializedMessage = doCustomSerialization(entity); return serializedMessage; } private byte[] doCustomSerialization(String entity) { return entity.getBytes(); } } {code} Here is also GitHub version https://github.com/soulfly/Kafka-java-producer So it just hungs on next line: {code} messagesProducer.send(data) {code} When I replaced the brokerlist to {code} props.put(metadata.broker.list, localhost:9092); {code} then I got an exception: {code} kafka.common.FailedToSendMessageException: Failed to send messages after 3 tries. {code} so it's okay Why it hungs with wrong brokerlist? Any ideas? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1951) Consumer offset checker should handle corner cases for dual commit or no offsets.
Jiangjie Qin created KAFKA-1951: --- Summary: Consumer offset checker should handle corner cases for dual commit or no offsets. Key: KAFKA-1951 URL: https://issues.apache.org/jira/browse/KAFKA-1951 Project: Kafka Issue Type: Bug Reporter: Jiangjie Qin Currently, if consumer offsets exists in Kafka, then offset checker assume that is the latest offsets and skip checking zookeeper. It should print the offsets in both place. When offset checker checks the offsets for not existing consumer group. It throws exception complaining about zk path does not exist. It should response with offset N/A. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-1950) Expose methods from SerializationTestUtils to all tests
Gwen Shapira created KAFKA-1950: --- Summary: Expose methods from SerializationTestUtils to all tests Key: KAFKA-1950 URL: https://issues.apache.org/jira/browse/KAFKA-1950 Project: Kafka Issue Type: Improvement Reporter: Gwen Shapira There are some super-useful setup methods in SerializationTestUtils. They can be used to make a lot of our test code simpler to write and read. I suggest pulling them into a more general test utils object (outside the api package) and exposing more of the methods as public. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1835: Fix Version/s: (was: 0.8.2.0) 0.8.3 Kafka new producer needs options to make blocking behavior explicit --- Key: KAFKA-1835 URL: https://issues.apache.org/jira/browse/KAFKA-1835 Project: Kafka Issue Type: Improvement Components: clients Affects Versions: 0.8.2.0, 0.8.3, 0.9.0 Reporter: Paul Pearcy Fix For: 0.8.3 Attachments: KAFKA-1835-New-producer--blocking_v0.patch, KAFKA-1835.patch Original Estimate: 504h Remaining Estimate: 504h The new (0.8.2 standalone) producer will block the first time it attempts to retrieve metadata for a topic. This is not the desired behavior in some use cases where async non-blocking guarantees are required and message loss is acceptable in known cases. Also, most developers will assume an API that returns a future is safe to call in a critical request path. Discussing on the mailing list, the most viable option is to have the following settings: pre.initialize.topics=x,y,z pre.initialize.timeout=x This moves potential blocking to the init of the producer and outside of some random request. The potential will still exist for blocking in a corner case where connectivity with Kafka is lost and a topic not included in pre-init has a message sent for the first time. There is the question of what to do when initialization fails. There are a couple of options that I'd like available: - Fail creation of the client - Fail all sends until the meta is available Open to input on how the above option should be expressed. It is also worth noting more nuanced solutions exist that could work without the extra settings, they just end up having extra complications and at the end of the day not adding much value. For instance, the producer could accept and queue messages(note: more complicated than I am making it sound due to storing all accepted messages in pre-partitioned compact binary form), but you're still going to be forced to choose to either start blocking or dropping messages at some point. I have some test cases I am going to port over to the Kafka producer integration ones and start from there. My current impl is in scala, but porting to Java shouldn't be a big deal (was using a promise to track init status, but will likely need to make that an atomic bool). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1897) Enhance MockProducer for more sophisticated tests
[ https://issues.apache.org/jira/browse/KAFKA-1897?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1897: Fix Version/s: (was: 0.8.2.0) 0.8.3 Enhance MockProducer for more sophisticated tests - Key: KAFKA-1897 URL: https://issues.apache.org/jira/browse/KAFKA-1897 Project: Kafka Issue Type: Bug Components: producer Reporter: Navina Ramesh Assignee: Jun Rao Fix For: 0.8.3 Based on the experience of upgrading the kafka producer in Samza, we faced two main constraints when using MockProducer: 1. The constructor requires a cluster specification and the tools to create a test cluster is not exposed. It is available from TestUtils in Kafka, however that jar is not published. This issue is currently being addressed in KAFKA-1861. 2. No support for testing a blocking client call. For example, flush in Samza blocks on the future returned by the latest send request. In order to test this, the MockProducer which buffers it should run in a concurrent mode. There is currently no provision to do this. We want the MockProducer to buffer the send and then, complete the callback concurrently while we wait for flush to unblock. We can write unit tests that have improved coverage if we can add support for concurrent execution of the MockProducer and unit test thread. For example implementation, please refer to the latest version of KafkaSystemProducer.scala in the Apache Samza repository. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1855) Topic unusable after unsuccessful UpdateMetadataRequest
[ https://issues.apache.org/jira/browse/KAFKA-1855?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1855: Fix Version/s: (was: 0.8.2.0) 0.8.3 Topic unusable after unsuccessful UpdateMetadataRequest --- Key: KAFKA-1855 URL: https://issues.apache.org/jira/browse/KAFKA-1855 Project: Kafka Issue Type: Bug Components: controller Affects Versions: 0.8.2.0 Reporter: Henri Pihkala Fix For: 0.8.3 Sometimes, seemingly randomly, topic creation/initialization might fail with the following lines in controller.log. Other logs show no errors. When this happens, the topic is unusable (gives UnknownTopicOrPartition for all requests). For me this happens 5-10% of the time. Feels like it's more likely to happen if there is time between topic creations. Observed on 0.8.2-beta, have not tried previous versions. [2015-01-09 16:15:27,153] WARN [Controller-0-to-broker-0-send-thread], Controller 0 fails to send a request to broker id:0,host:192.168.10.21,port:9092 (kafka.controller.RequestSendThread) java.io.EOFException: Received -1 when reading from channel, socket has likely been closed. at kafka.utils.Utils$.read(Utils.scala:381) at kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54) at kafka.network.Receive$class.readCompletely(Transmission.scala:56) at kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29) at kafka.network.BlockingChannel.receive(BlockingChannel.scala:108) at kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:146) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60) [2015-01-09 16:15:27,156] ERROR [Controller-0-to-broker-0-send-thread], Controller 0 epoch 6 failed to send request Name:UpdateMetadataRequest;Version:0;Controller:0;ControllerEpoch:6;CorrelationId:48;ClientId:id_0-host_192.168.10.21-port_9092;AliveBrokers:id:0,host:192.168.10.21,port:9092;PartitionState:[40963064-cdd2-4cd1-937a-9827d3ab77ad,0] - (LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEpoch:6),ReplicationFactor:1),AllReplicas:0) to broker id:0,host:192.168.10.21,port:9092. Reconnecting to broker. (kafka.controller.RequestSendThread) java.nio.channels.ClosedChannelException at kafka.network.BlockingChannel.send(BlockingChannel.scala:97) at kafka.controller.RequestSendThread.liftedTree1$1(ControllerChannelManager.scala:132) at kafka.controller.RequestSendThread.doWork(ControllerChannelManager.scala:131) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1282) Disconnect idle socket connection in Selector
[ https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1282: Fix Version/s: (was: 0.8.2.0) 0.8.3 Disconnect idle socket connection in Selector - Key: KAFKA-1282 URL: https://issues.apache.org/jira/browse/KAFKA-1282 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.8.2.0 Reporter: Jun Rao Assignee: nicu marasoiu Labels: newbie++ Fix For: 0.8.3 Attachments: 1282_access-order.patch, 1282_brush.patch, 1282_brushed_up.patch, KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch To reduce # socket connections, it would be useful for the new producer to close socket connections that are idle. We can introduce a new producer config for the idle time. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1724) Errors after reboot in single node setup
[ https://issues.apache.org/jira/browse/KAFKA-1724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1724: Fix Version/s: (was: 0.8.2.0) 0.8.3 Errors after reboot in single node setup Key: KAFKA-1724 URL: https://issues.apache.org/jira/browse/KAFKA-1724 Project: Kafka Issue Type: Bug Affects Versions: 0.8.2.0 Reporter: Ciprian Hacman Assignee: Sriharsha Chintalapani Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1724.patch In a single node setup, after reboot, Kafka logs show the following: {code} [2014-10-22 16:37:22,206] INFO [Controller 0]: Controller starting up (kafka.controller.KafkaController) [2014-10-22 16:37:22,419] INFO [Controller 0]: Controller startup complete (kafka.controller.KafkaController) [2014-10-22 16:37:22,554] INFO conflict in /brokers/ids/0 data: {jmx_port:-1,timestamp:1413995842465,host:ip-10-91-142-54.eu-west-1.compute.internal,version:1,port:9092} stored data: {jmx_port:-1,timestamp:1413994171579,host:ip-10-91-142-54.eu-west-1.compute.internal,version:1,port:9092} (kafka.utils.ZkUtils$) [2014-10-22 16:37:22,736] INFO I wrote this conflicted ephemeral node [{jmx_port:-1,timestamp:1413995842465,host:ip-10-91-142-54.eu-west-1.compute.internal,version:1,port:9092}] at /brokers/ids/0 a while back in a different session, hence I will backoff for this node to be deleted by Zookeeper and retry (kafka.utils.ZkUtils$) [2014-10-22 16:37:25,010] ERROR Error handling event ZkEvent[Data of /controller changed sent to kafka.server.ZookeeperLeaderElector$LeaderChangeListener@a6af882] (org.I0Itec.zkclient.ZkEventThread) java.lang.IllegalStateException: Kafka scheduler has not been started at kafka.utils.KafkaScheduler.ensureStarted(KafkaScheduler.scala:114) at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:86) at kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:350) at kafka.controller.KafkaController$$anonfun$2.apply$mcV$sp(KafkaController.scala:162) at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:138) at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:134) at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:134) at kafka.utils.Utils$.inLock(Utils.scala:535) at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:134) at org.I0Itec.zkclient.ZkClient$6.run(ZkClient.java:549) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) [2014-10-22 16:37:28,757] INFO Registered broker 0 at path /brokers/ids/0 with address ip-10-91-142-54.eu-west-1.compute.internal:9092. (kafka.utils.ZkUtils$) [2014-10-22 16:37:28,849] INFO [Kafka Server 0], started (kafka.server.KafkaServer) [2014-10-22 16:38:56,718] INFO Closing socket connection to /127.0.0.1. (kafka.network.Processor) [2014-10-22 16:38:56,850] INFO Closing socket connection to /127.0.0.1. (kafka.network.Processor) [2014-10-22 16:38:56,985] INFO Closing socket connection to /127.0.0.1. (kafka.network.Processor) {code} The last log line repeats forever and is correlated with errors on the app side. Restarting Kafka fixes the errors. Steps to reproduce (with help from the mailing list): # start zookeeper # start kafka-broker # create topic or start a producer writing to a topic # stop zookeeper # stop kafka-broker( kafka broker shutdown goes into WARN Session 0x14938d9dc010001 for server null, unexpected error, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn) java.net.ConnectException: Connection refused) # kill -9 kafka-broker # restart zookeeper and than kafka-broker leads into the the error above -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
+1 on investigating it further as a separate feature that will improve ops significantly (especially since an expert on the operations side has described use cases from actual experience). On Thu, Feb 12, 2015 at 05:47:50PM -0800, Gwen Shapira wrote: I REALLY like the idea of supporting separate network for inter-broker communication (and probably Zookeeper too). I think its actually a pretty typical configuration in clusters, so I'm surprised we didn't think of it before :) Servers arrive with multiple cards specifically for admin nic vs. clients nic vs storage nic. That said, I'd like to handle it in a separate patch. First because KAFKA-1809 is big enough already, and second because this really deserve its own requirement-gathering and design. Does that make sense? Gwen On Thu, Feb 12, 2015 at 12:34 PM, Todd Palino tpal...@gmail.com wrote: The idea is more about isolating the intra-cluster traffic from the normal clients as much as possible. There's a couple situations we've seen where this would be useful that I can think of immediately: 1) Normal operation - just having the intra-cluster traffic on a separate network interface would allow it to not get overwhelmed by something like a bootstrapping client who is saturating the network interface. We see this fairly often, where the replication falls behind because of heavy traffic from one application. We can always adjust the network threads, but segregating the traffic is the first step. 2) Isolation in case of an error - We have had situations, more than once, where we are needing to rebuild a cluster after a catastrophic problem and the clients are causing that process to take too long, or are causing additional failures. This has mostly come into play with file descriptor limits in the past, but it's certainly not the only situation. Constantly reconnecting clients continue to cause the brokers to fall over while we are trying to recover a down cluster. The only solution was to firewall off all the clients temporarily. This is a great deal more complicated if the brokers and the clients are all operating over the same port. Now, that said, quotas can be a partial solution to this. I don't want to jump the gun on that discussion (because it's going to come up separately and in more detail), but it is possible to structure quotas in a way that will allow the intra-cluster replication to continue to function in the case of high load. That would partially address case 1, but it does nothing for case 2. Additionally, I think it is also desirable to segregate the traffic even with quotas, so that regardless of the client load, the cluster itself is able to be healthy. -Todd On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version back to 0.8.2 and do a rolling bounce? On the host/port/protocol specification, specifically the ZK config format, is it possible to have an un-advertised endpoint? I would see this as potentially useful if you wanted to have an endpoint that you are reserving for intra-cluster communication, and you would prefer to not have it advertised at all. Perhaps it is
[jira] [Updated] (KAFKA-1697) remove code related to ack1 on the broker
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1697: Attachment: KAFKA-1697_2015-02-12_18:57:36.patch remove code related to ack1 on the broker -- Key: KAFKA-1697 URL: https://issues.apache.org/jira/browse/KAFKA-1697 Project: Kafka Issue Type: Bug Reporter: Jun Rao Assignee: Gwen Shapira Fix For: 0.8.3 Attachments: KAFKA-1697.patch, KAFKA-1697_2015-01-14_15:41:37.patch, KAFKA-1697_2015-02-10_17:06:51.patch, KAFKA-1697_2015-02-11_18:45:42.patch, KAFKA-1697_2015-02-11_18:47:53.patch, KAFKA-1697_2015-02-11_23:13:53.patch, KAFKA-1697_2015-02-12_18:57:36.patch We removed the ack1 support from the producer client in kafka-1555. We can completely remove the code in the broker that supports ack1. Also, we probably want to make NotEnoughReplicasAfterAppend a non-retriable exception and let the client decide what to do. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29647: Patch for KAFKA-1697
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 13, 2015, 2:57 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description (updated) --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity KAFKA-1948; Fix ConsumerTest.testPartitionReassignmentCallback handling issue; reviewed by Gwen Shapira Merge branch 'trunk' of http://git-wip-us.apache.org/repos/asf/kafka into KAFKA-1697 improved readability of append rules Diffs (updated) - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/controller/KafkaController.scala 66df6d2fbdbdd556da6bea0df84f93e0472c8fbf core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/integration/kafka/api/ConsumerTest.scala 798f035df52e405176f558806584ce25e8c392ac core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
[jira] [Commented] (KAFKA-1697) remove code related to ack1 on the broker
[ https://issues.apache.org/jira/browse/KAFKA-1697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14319498#comment-14319498 ] Gwen Shapira commented on KAFKA-1697: - Updated reviewboard https://reviews.apache.org/r/29647/diff/ against branch trunk remove code related to ack1 on the broker -- Key: KAFKA-1697 URL: https://issues.apache.org/jira/browse/KAFKA-1697 Project: Kafka Issue Type: Bug Reporter: Jun Rao Assignee: Gwen Shapira Fix For: 0.8.3 Attachments: KAFKA-1697.patch, KAFKA-1697_2015-01-14_15:41:37.patch, KAFKA-1697_2015-02-10_17:06:51.patch, KAFKA-1697_2015-02-11_18:45:42.patch, KAFKA-1697_2015-02-11_18:47:53.patch, KAFKA-1697_2015-02-11_23:13:53.patch, KAFKA-1697_2015-02-12_18:57:36.patch We removed the ack1 support from the producer client in kafka-1555. We can completely remove the code in the broker that supports ack1. Also, we probably want to make NotEnoughReplicasAfterAppend a non-retriable exception and let the client decide what to do. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Build failed in Jenkins: Kafka-trunk #391
See https://builds.apache.org/job/Kafka-trunk/391/changes Changes: [wangguoz] KAFKA-1948; Fix ConsumerTest.testPartitionReassignmentCallback handling issue; reviewed by Gwen Shapira -- [...truncated 1209 lines...] kafka.log.LogSegmentTest testRecoveryFixesCorruptIndex PASSED kafka.log.LogSegmentTest testRecoveryWithCorruptMessage PASSED kafka.log.LogManagerTest testCreateLog PASSED kafka.log.LogManagerTest testGetNonExistentLog PASSED kafka.log.LogManagerTest testCleanupExpiredSegments PASSED kafka.log.LogManagerTest testCleanupSegmentsToMaintainSize PASSED kafka.log.LogManagerTest testTimeBasedFlush PASSED kafka.log.LogManagerTest testLeastLoadedAssignment PASSED kafka.log.LogManagerTest testTwoLogManagersUsingSameDirFails PASSED kafka.log.LogManagerTest testCheckpointRecoveryPoints PASSED kafka.log.LogManagerTest testRecoveryDirectoryMappingWithTrailingSlash PASSED kafka.log.LogManagerTest testRecoveryDirectoryMappingWithRelativeDirectory PASSED kafka.log.CleanerTest testCleanSegments PASSED kafka.log.CleanerTest testCleaningWithDeletes PASSED kafka.log.CleanerTest testCleanSegmentsWithAbort PASSED kafka.log.CleanerTest testSegmentGrouping PASSED kafka.log.CleanerTest testBuildOffsetMap PASSED kafka.log.OffsetIndexTest truncate PASSED kafka.log.OffsetIndexTest randomLookupTest PASSED kafka.log.OffsetIndexTest lookupExtremeCases PASSED kafka.log.OffsetIndexTest appendTooMany PASSED kafka.log.OffsetIndexTest appendOutOfOrder PASSED kafka.log.OffsetIndexTest testReopen PASSED kafka.log.LogConfigTest testFromPropsDefaults PASSED kafka.log.LogConfigTest testFromPropsEmpty PASSED kafka.log.LogConfigTest testFromPropsToProps PASSED kafka.log.LogConfigTest testFromPropsInvalid PASSED kafka.log.OffsetMapTest testBasicValidation PASSED kafka.log.OffsetMapTest testClear PASSED kafka.log.LogCleanerIntegrationTest cleanerTest PASSED kafka.log.LogTest testTimeBasedLogRoll PASSED kafka.log.LogTest testTimeBasedLogRollJitter PASSED kafka.log.LogTest testSizeBasedLogRoll PASSED kafka.log.LogTest testLoadEmptyLog PASSED kafka.log.LogTest testAppendAndReadWithSequentialOffsets PASSED kafka.log.LogTest testAppendAndReadWithNonSequentialOffsets PASSED kafka.log.LogTest testReadAtLogGap PASSED kafka.log.LogTest testReadOutOfRange PASSED kafka.log.LogTest testLogRolls PASSED kafka.log.LogTest testCompressedMessages PASSED kafka.log.LogTest testThatGarbageCollectingSegmentsDoesntChangeOffset PASSED kafka.log.LogTest testMessageSetSizeCheck PASSED kafka.log.LogTest testMessageSizeCheck PASSED kafka.log.LogTest testLogRecoversToCorrectOffset PASSED kafka.log.LogTest testIndexRebuild PASSED kafka.log.LogTest testTruncateTo PASSED kafka.log.LogTest testIndexResizingAtTruncation PASSED kafka.log.LogTest testBogusIndexSegmentsAreRemoved PASSED kafka.log.LogTest testReopenThenTruncate PASSED kafka.log.LogTest testAsyncDelete PASSED kafka.log.LogTest testOpenDeletesObsoleteFiles PASSED kafka.log.LogTest testAppendMessageWithNullPayload PASSED kafka.log.LogTest testCorruptLog PASSED kafka.log.LogTest testCleanShutdownFile PASSED kafka.log.LogTest testParseTopicPartitionName PASSED kafka.log.LogTest testParseTopicPartitionNameForEmptyName PASSED kafka.log.LogTest testParseTopicPartitionNameForNull PASSED kafka.log.LogTest testParseTopicPartitionNameForMissingSeparator PASSED kafka.log.LogTest testParseTopicPartitionNameForMissingTopic PASSED kafka.log.LogTest testParseTopicPartitionNameForMissingPartition PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[0] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[1] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[2] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[3] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[4] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[5] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[6] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[7] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[8] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[9] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[10] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[11] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[12] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[13] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[14] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[15] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[16] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[17] PASSED kafka.log.BrokerCompressionTest testBrokerSideCompression[18] PASSED kafka.log.BrokerCompressionTest
Re: Review Request 29647: Patch for KAFKA-1697
On Feb. 12, 2015, 2:31 p.m., Joel Koshy wrote: core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala, line 62 https://reviews.apache.org/r/29647/diff/6/?file=861626#file861626line62 Is this change necessary? Gwen Shapira wrote: Nope, but I need to change the test to use produceRequest from TestUtils and not the API test. It snack in when I moved the test code around. Let me fix this. Actually, the methods from SerializationTestUtils are super useful, and its a bit of a bummer that the are private and part of RequestResponseSerializationTest. Do you mind leaving this as is and I'll open a separate JIRA to refactor those test utils into something reusable? - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72156 --- On Feb. 12, 2015, 7:14 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 7:14 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
Re: Review Request 29647: Patch for KAFKA-1697
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72317 --- Ship it! KAFKA-1948 changes crept in to the rb, but I can clean that up. - Joel Koshy On Feb. 13, 2015, 2:57 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 13, 2015, 2:57 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity KAFKA-1948; Fix ConsumerTest.testPartitionReassignmentCallback handling issue; reviewed by Gwen Shapira Merge branch 'trunk' of http://git-wip-us.apache.org/repos/asf/kafka into KAFKA-1697 improved readability of append rules Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/controller/KafkaController.scala 66df6d2fbdbdd556da6bea0df84f93e0472c8fbf core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/integration/kafka/api/ConsumerTest.scala 798f035df52e405176f558806584ce25e8c392ac core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
Re: [DISCUSSION] KIP-2: Refactor Brokers to Allow Multiple Endpoints
I'll be happy to give the initial design a go, but will probably only get to it after Strata. So either wait a bit (there are enough KIPs to review ;) or someone else can get started. Gwen On Thu, Feb 12, 2015 at 6:55 PM, Joel Koshy jjkosh...@gmail.com wrote: +1 on investigating it further as a separate feature that will improve ops significantly (especially since an expert on the operations side has described use cases from actual experience). On Thu, Feb 12, 2015 at 05:47:50PM -0800, Gwen Shapira wrote: I REALLY like the idea of supporting separate network for inter-broker communication (and probably Zookeeper too). I think its actually a pretty typical configuration in clusters, so I'm surprised we didn't think of it before :) Servers arrive with multiple cards specifically for admin nic vs. clients nic vs storage nic. That said, I'd like to handle it in a separate patch. First because KAFKA-1809 is big enough already, and second because this really deserve its own requirement-gathering and design. Does that make sense? Gwen On Thu, Feb 12, 2015 at 12:34 PM, Todd Palino tpal...@gmail.com wrote: The idea is more about isolating the intra-cluster traffic from the normal clients as much as possible. There's a couple situations we've seen where this would be useful that I can think of immediately: 1) Normal operation - just having the intra-cluster traffic on a separate network interface would allow it to not get overwhelmed by something like a bootstrapping client who is saturating the network interface. We see this fairly often, where the replication falls behind because of heavy traffic from one application. We can always adjust the network threads, but segregating the traffic is the first step. 2) Isolation in case of an error - We have had situations, more than once, where we are needing to rebuild a cluster after a catastrophic problem and the clients are causing that process to take too long, or are causing additional failures. This has mostly come into play with file descriptor limits in the past, but it's certainly not the only situation. Constantly reconnecting clients continue to cause the brokers to fall over while we are trying to recover a down cluster. The only solution was to firewall off all the clients temporarily. This is a great deal more complicated if the brokers and the clients are all operating over the same port. Now, that said, quotas can be a partial solution to this. I don't want to jump the gun on that discussion (because it's going to come up separately and in more detail), but it is possible to structure quotas in a way that will allow the intra-cluster replication to continue to function in the case of high load. That would partially address case 1, but it does nothing for case 2. Additionally, I think it is also desirable to segregate the traffic even with quotas, so that regardless of the client load, the cluster itself is able to be healthy. -Todd On Thu, Feb 12, 2015 at 11:38 AM, Jun Rao j...@confluent.io wrote: Todd, Could you elaborate on the benefit for having a separate endpoint for intra-cluster communication? Is it mainly for giving intra-cluster requests a high priority? At this moment, having a separate endpoint just means that the socket connection for the intra-cluster communication is handled by a separate acceptor thread. The processing of the requests from the network and the handling of the requests are still shared by a single thread pool. So, if any of the thread pool is exhausted, the intra-cluster requests will still be delayed. We can potentially change this model, but this requires more work. An alternative is to just rely on quotas. Intra-cluster requests will be exempt from any kind of throttling. Gwen, I agree that defaulting wire.protocol.version to the current version is probably better. It just means that we need to document the migration path for previous versions. Thanks, Jun On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote: Thanks, Gwen. This looks good to me as far as the wire protocol versioning goes. I agree with you on defaulting to the new wire protocol version for new installs. I think it will also need to be very clear (to general installer of Kafka, and not just developers) in documentation when the wire protocol version changes moving forwards, and what the risk/benefit of changing to the new version is. Since a rolling upgrade of the intra-cluster protocol is supported, will a rolling downgrade work as well? Should a flaw (bug, security, or otherwise) be discovered after upgrade, is it possible to change the wire.protocol.version
Re: Review Request 29647: Patch for KAFKA-1697
On Feb. 13, 2015, 3:05 a.m., Joel Koshy wrote: KAFKA-1948 changes crept in to the rb, but I can clean that up. Thanks, man :) I think I need lessons on how to do merges and use the patch-review tool... - Gwen --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72317 --- On Feb. 13, 2015, 2:57 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 13, 2015, 2:57 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity KAFKA-1948; Fix ConsumerTest.testPartitionReassignmentCallback handling issue; reviewed by Gwen Shapira Merge branch 'trunk' of http://git-wip-us.apache.org/repos/asf/kafka into KAFKA-1697 improved readability of append rules Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/controller/KafkaController.scala 66df6d2fbdbdd556da6bea0df84f93e0472c8fbf core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/integration/kafka/api/ConsumerTest.scala 798f035df52e405176f558806584ce25e8c392ac core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
Re: gradle testAll stuck
- Can you enable test logging (see the README) and see if you can figure out which test is getting stuck or taking forever? - A thread-dump may help. On Thu, Feb 12, 2015 at 08:57:11AM -0500, Tong Li wrote: Hi, folks, How are you all doing? New bee here. Run gradle --daemon testAll on 0.8.2 worked and finished in about 30 minutes. pulled down trunk and run samething, always stuck. left it run for overnight, checked in the morning still stuck. even on 0.8.2, it still takes over 30 minutes on my modest dev system. Wonder how you all run gradle and is there any specific settings needed? I am running it on Ubuntu 14.04. Any help or pointer is really appreciated. Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com
Re: Review Request 29647: Patch for KAFKA-1697
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72156 --- Ship it! Thanks Gwen - a few minor comments that I can fix up - can you comment on these? core/src/main/scala/kafka/server/ReplicaManager.scala https://reviews.apache.org/r/29647/#comment118159 This is good, but maybe call this canRespondNow ? Since before replication won't apply if there are no errors in the partition appends or if there is no data. core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala https://reviews.apache.org/r/29647/#comment118160 Is this change necessary? - Joel Koshy On Feb. 12, 2015, 7:14 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 7:14 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
Re: gradle testAll stuck
This may be due to the recent issue reported by Gwen. https://issues.apache.org/jira/browse/KAFKA-1948 On 2/12/15, Joel Koshy jjkosh...@gmail.com wrote: - Can you enable test logging (see the README) and see if you can figure out which test is getting stuck or taking forever? - A thread-dump may help. On Thu, Feb 12, 2015 at 08:57:11AM -0500, Tong Li wrote: Hi, folks, How are you all doing? New bee here. Run gradle --daemon testAll on 0.8.2 worked and finished in about 30 minutes. pulled down trunk and run samething, always stuck. left it run for overnight, checked in the morning still stuck. even on 0.8.2, it still takes over 30 minutes on my modest dev system. Wonder how you all run gradle and is there any specific settings needed? I am running it on Ubuntu 14.04. Any help or pointer is really appreciated. Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com
gradle testAll stuck
Hi, folks, How are you all doing? New bee here. Run gradle --daemon testAll on 0.8.2 worked and finished in about 30 minutes. pulled down trunk and run samething, always stuck. left it run for overnight, checked in the morning still stuck. even on 0.8.2, it still takes over 30 minutes on my modest dev system. Wonder how you all run gradle and is there any specific settings needed? I am running it on Ubuntu 14.04. Any help or pointer is really appreciated. Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com
Re: gradle testAll stuck
Manikumar, yes, exactly the same issue. Commented on https://issues.apache.org/jira/browse/KAFKA-1948. Before this problem happens, the testAll normally still takes about half hour? Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com From: Manikumar Reddy ku...@nmsworks.co.in To: dev@kafka.apache.org Date: 02/12/2015 09:10 AM Subject:Re: gradle testAll stuck Sent by:manikumar.re...@gmail.com This may be due to the recent issue reported by Gwen. https://issues.apache.org/jira/browse/KAFKA-1948 On 2/12/15, Joel Koshy jjkosh...@gmail.com wrote: - Can you enable test logging (see the README) and see if you can figure out which test is getting stuck or taking forever? - A thread-dump may help. On Thu, Feb 12, 2015 at 08:57:11AM -0500, Tong Li wrote: Hi, folks, How are you all doing? New bee here. Run gradle --daemon testAll on 0.8.2 worked and finished in about 30 minutes. pulled down trunk and run samething, always stuck. left it run for overnight, checked in the morning still stuck. even on 0.8.2, it still takes over 30 minutes on my modest dev system. Wonder how you all run gradle and is there any specific settings needed? I am running it on Ubuntu 14.04. Any help or pointer is really appreciated. Thanks Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.com
Re: Review Request 29912: Patch for KAFKA-1852
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29912/#review72166 --- core/src/main/scala/kafka/server/OffsetManager.scala https://reviews.apache.org/r/29912/#comment118171 Can we just add an exists(topic) method to metadataCache? That way we can just do something like offsetMetadata.groupBy((topicPartition, offsetMetadata) = metadataCache.contains(topicPartition.topic)) - Joel Koshy On Jan. 19, 2015, 6:44 p.m., Sriharsha Chintalapani wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29912/ --- (Updated Jan. 19, 2015, 6:44 p.m.) Review request for kafka. Bugs: KAFKA-1852 https://issues.apache.org/jira/browse/KAFKA-1852 Repository: kafka Description --- KAFKA-1852. OffsetCommitRequest can commit offset on unknown topic. Diffs - core/src/main/scala/kafka/server/KafkaApis.scala ec8d9f7ba44741db40875458bd524c4062ad6a26 core/src/main/scala/kafka/server/OffsetManager.scala 0bdd42fea931cddd072c0fff765b10526db6840a core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 5b93239cdc26b5be7696f4e7863adb9fbe5f0ed5 Diff: https://reviews.apache.org/r/29912/diff/ Testing --- Thanks, Sriharsha Chintalapani
[jira] [Commented] (KAFKA-1887) controller error message on shutting the last broker
[ https://issues.apache.org/jira/browse/KAFKA-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318312#comment-14318312 ] Sriharsha Chintalapani commented on KAFKA-1887: --- [~gwenshap] I am worried about moving kafkaHealthCheck below as it won't be able to trigger onBrokerFailure in KafkaController. In a cluster this probably fine as there will be another controller gets elected. Also this change causes ProducerFailureHandlingTest.testNotEnoughReplicasAfterBrokerShutdown to fail as producer.send.get gets an NotEnoughReplicasAfterAppendException instead of NotEnoughReplicasException controller error message on shutting the last broker Key: KAFKA-1887 URL: https://issues.apache.org/jira/browse/KAFKA-1887 Project: Kafka Issue Type: Bug Components: core Reporter: Jun Rao Assignee: Sriharsha Chintalapani Priority: Minor Fix For: 0.8.3 We always see the following error in state-change log on shutting down the last broker. [2015-01-20 13:21:04,036] ERROR Controller 0 epoch 3 initiated state change for partition [test,0] from OfflinePartition to OnlinePartition failed (state.change.logger) kafka.common.NoReplicaOnlineException: No replica for partition [test,0] is alive. Live brokers are: [Set()], Assigned replicas are: [List(0)] at kafka.controller.OfflinePartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:75) at kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:357) at kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:206) at kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:120) at kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:117) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771) at kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:117) at kafka.controller.KafkaController.onBrokerFailure(KafkaController.scala:446) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:373) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:359) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:359) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(ReplicaStateMachine.scala:358) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:357) at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:357) at kafka.utils.Utils$.inLock(Utils.scala:535) at kafka.controller.ReplicaStateMachine$BrokerChangeListener.handleChildChange(ReplicaStateMachine.scala:356) at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1948) kafka.api.consumerTests are hanging
[ https://issues.apache.org/jira/browse/KAFKA-1948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318311#comment-14318311 ] Tong Li commented on KAFKA-1948: Pulled down the latest trunk and run into this problem. It will be nice if someone can take a look. The test does not stuck always at the same place. I did notice there are tons of threads getting created and died. What I did is when the test hung, I ran this command. top -H -p theprocessid of the test java You will notice that many threads come and go. and the state of these threads are always S (meaning sleep). including the parent thread. Not really sure what the deal is. kafka.api.consumerTests are hanging --- Key: KAFKA-1948 URL: https://issues.apache.org/jira/browse/KAFKA-1948 Project: Kafka Issue Type: Bug Reporter: Gwen Shapira Noticed today that very often when I run the full test suite, it hangs on kafka.api.consumerTest (not always same test though). It doesn't reproduce 100% of the time, but enough to be very annoying. I also saw it happening on trunk after KAFKA-1333: https://builds.apache.org/view/All/job/Kafka-trunk/389/ -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1938) [doc] Quick start example should reference appropriate Kafka version
[ https://issues.apache.org/jira/browse/KAFKA-1938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy updated KAFKA-1938: --- Attachment: remove-081-references-1.patch uploading new patch with small correction. [doc] Quick start example should reference appropriate Kafka version Key: KAFKA-1938 URL: https://issues.apache.org/jira/browse/KAFKA-1938 Project: Kafka Issue Type: Improvement Components: website Affects Versions: 0.8.2.0 Reporter: Stevo Slavic Assignee: Manikumar Reddy Priority: Trivial Attachments: remove-081-references-1.patch, remove-081-references.patch Kafka 0.8.2.0 documentation, quick start example on https://kafka.apache.org/documentation.html#quickstart in step 1 links and instructs reader to download Kafka 0.8.1.1. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 29647: Patch for KAFKA-1697
On Feb. 12, 2015, 2:31 p.m., Joel Koshy wrote: core/src/main/scala/kafka/server/ReplicaManager.scala, line 271 https://reviews.apache.org/r/29647/diff/6/?file=861625#file861625line271 This is good, but maybe call this canRespondNow ? Since before replication won't apply if there are no errors in the partition appends or if there is no data. In fact flipping the condition and calling the helper function delayedRequestRequired may be even clearer - Joel --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/#review72156 --- On Feb. 12, 2015, 7:14 a.m., Gwen Shapira wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29647/ --- (Updated Feb. 12, 2015, 7:14 a.m.) Review request for kafka. Bugs: KAFKA-1697 https://issues.apache.org/jira/browse/KAFKA-1697 Repository: kafka Description --- added early handling of invalid number of acks to handler and a test merging with current trunk moved check for legal requiredAcks to append and fixed the tests accordingly changing exception back to retriable cleaning unused exceptions refactored appendToLog for clarity Diffs - clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java a6107b818947d6d6818c85cdffcb2b13f69a55c0 clients/src/main/java/org/apache/kafka/common/protocol/Errors.java a8deac4ce5149129d0a6f44c0526af9d55649a36 core/src/main/scala/kafka/cluster/Partition.scala e6ad8be5e33b6fb31c078ad78f8de709869ddc04 core/src/main/scala/kafka/server/KafkaApis.scala 6ee7d8819a9ef923f3a65c865a0a3d8ded8845f0 core/src/main/scala/kafka/server/ReplicaManager.scala fb948b9ab28c516e81dab14dcbe211dcd99842b6 core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala a1f72f8c2042ff2a43af503b2e06f84706dad9db core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala faa907131ed0aa94a7eacb78c1ffb576062be87a Diff: https://reviews.apache.org/r/29647/diff/ Testing --- Thanks, Gwen Shapira
[jira] [Commented] (KAFKA-1711) WARN Property topic is not valid when running console producer
[ https://issues.apache.org/jira/browse/KAFKA-1711?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14318458#comment-14318458 ] Jelmer Kuperus commented on KAFKA-1711: --- Issue should be reopened, the fix got reverted as part of https://issues.apache.org/jira/browse/KAFKA-1824 WARN Property topic is not valid when running console producer -- Key: KAFKA-1711 URL: https://issues.apache.org/jira/browse/KAFKA-1711 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.2.0 Reporter: Jun Rao Assignee: Joe Crobak Labels: newbie Fix For: 0.8.3 Attachments: KAFKA-1711.patch bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test [2014-10-17 09:54:23,984] WARN Property topic is not valid (kafka.utils.VerifiableProperties) It would be good if we can get rid of the warning. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14317824#comment-14317824 ] Honghai Chen commented on KAFKA-1947: - Created reviewboard against branch origin/trunk can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-1947) can't explicitly set replica-assignment when add partitions
[ https://issues.apache.org/jira/browse/KAFKA-1947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Honghai Chen updated KAFKA-1947: Attachment: KAFKA-1947.patch can't explicitly set replica-assignment when add partitions --- Key: KAFKA-1947 URL: https://issues.apache.org/jira/browse/KAFKA-1947 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.8.1.1 Environment: Windows Reporter: Honghai Chen Attachments: KAFKA-1947.patch, KAFKA-1947.patch, KAFKA-1947.patch When create topic, the replicaAssignmentOpt should not appear with partitions. But when add partitions, they should can appear together, from the code below, you can see when alter topic, and has partitions in arguments, it try get replica-assignment https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l114 The root cause is below code: CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, 305 allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) https://git1-us-west.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/admin/TopicCommand.scala;h=285c0333ff43543d3e46444c1cd9374bb883bb59;hb=HEAD#l304 Related: https://issues.apache.org/jira/browse/KAFKA-1052 -- This message was sent by Atlassian JIRA (v6.3.4#6332)