Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-07-25 Thread Ismael Juma
Hi Tom and Paolo, It's true that increasing the number of partitions is done via the kafka-topics tool, which is also being converted to use the AdminClient (but via a different JIRA). I also agree that it would be good to consider if alterTopics would be a sensible way to support all the use

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-07-25 Thread Tom Bentley
Hi Ismael, I assume that's the --partitions for kafka-topics.sh? I must admit I hadn't considered that tool, only kafka-reassign-partitions.sh. Thanks for pointing it out, because obviously the AdminClient API needs to be suitable for reuse in kafka-topics.sh too. Since AdminClient doesn't

Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Ismael Juma
On Tue, Jul 18, 2017 at 4:04 PM, Guozhang Wang wrote: > I was actually thinking about using dot as well for the rc as well moving > forward, but I can be convinced if we have some reason to keep it as dash > as well. It seems reasonable to use a dash for the RC part as it's

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-07-25 Thread Paolo Patierno
Hi, I was digging into it because I need something like an Admin Client alter API for my work on rewriting the TopicCommand tool using them. The AlterConfigs API is used for changing topic level configuration (i.e. retention.ms, retention.bytes and so on). A new AlterTopic API could be

[jira] [Created] (KAFKA-5635) KIP-181 Kafka-Connect integrate with kafka ReST Proxy

2017-07-25 Thread Dhananjay Patkar (JIRA)
Dhananjay Patkar created KAFKA-5635: --- Summary: KIP-181 Kafka-Connect integrate with kafka ReST Proxy Key: KAFKA-5635 URL: https://issues.apache.org/jira/browse/KAFKA-5635 Project: Kafka

[GitHub] kafka pull request #3572: MINOR: Remove unused GroupState.state field

2017-07-25 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3572 MINOR: Remove unused GroupState.state field This field doesn't seem to be used and the value for `AwaitingSync` seems to be wrong (it seems like it should have been `2` instead of `5`). You

Re: [DISCUSS] KIP-180: Add a broker metric specifying the number of consumer group rebalances in progress

2017-07-25 Thread Ismael Juma
Hi Guozhang, Thanks for the clarification. The naming does seem a bit unclear. Maybe `PreparingRebalance` could be `StartingRebalance` or something that makes it clear that it is part of the rebalance instead of a step before the actual rebalance. `AwaitingSync` could also be

[jira] [Created] (KAFKA-5636) DSL: allow sliding windows to be used directly (i.e. not just implicitly when doing joins)

2017-07-25 Thread Michael Noll (JIRA)
Michael Noll created KAFKA-5636: --- Summary: DSL: allow sliding windows to be used directly (i.e. not just implicitly when doing joins) Key: KAFKA-5636 URL: https://issues.apache.org/jira/browse/KAFKA-5636

Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Guozhang Wang
Makes sense, have updated the wiki page to use dashes for rc numbers. Guozhang On Tue, Jul 25, 2017 at 3:40 AM, Ismael Juma wrote: > On Tue, Jul 18, 2017 at 4:04 PM, Guozhang Wang wrote: > > > I was actually thinking about using dot as well for the rc as

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-07-25 Thread Paolo Patierno
Hi Tom, I haven taken a look to the updated KIP, some thoughts : * in the "Public Interfaces" section you wrote alterTopics(Set) but then a collection is used (instead of a set) in the Proposed Changes section. I'm ok with collection. * in the summary of the alterTopics method you

[jira] [Created] (KAFKA-5637) Document compatibility and release policies

2017-07-25 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5637: -- Summary: Document compatibility and release policies Key: KAFKA-5637 URL: https://issues.apache.org/jira/browse/KAFKA-5637 Project: Kafka Issue Type:

Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Ismael Juma
Thanks Sonke, that's great. I filed an initial JIRA: https://issues.apache.org/jira/browse/KAFKA-5637 As per our offline conversation, you captured the thread discussion already, so feel free to flesh out the JIRA. Ismael On Mon, Jul 24, 2017 at 8:45 PM, Sönke Liebau <

Re: [VOTE] KIP-167 (Addendum): Add interface for the state store restoration process

2017-07-25 Thread Sriram Subramanian
+1 On Fri, Jul 21, 2017 at 12:08 PM, Guozhang Wang wrote: > +1 > > On Thu, Jul 20, 2017 at 11:00 PM, Matthias J. Sax > wrote: > > > +1 > > > > On 7/20/17 4:22 AM, Bill Bejeck wrote: > > > Hi, > > > > > > After working on the PR for this KIP I

[jira] [Created] (KAFKA-5638) Inconsistency in consumer group related ACLs

2017-07-25 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-5638: -- Summary: Inconsistency in consumer group related ACLs Key: KAFKA-5638 URL: https://issues.apache.org/jira/browse/KAFKA-5638 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-175: Additional '--describe' views for ConsumerGroupCommand

2017-07-25 Thread Vahid S Hashemian
Hi, If there is no further feedback on this KIP, I'll start the vote tomorrow. Thanks. --Vahid From: Vahid S Hashemian/Silicon Valley/IBM To: dev , "Kafka User" Date: 07/03/2017 04:06 PM Subject:[DISCUSS] KIP-175: Additional

[GitHub] kafka pull request #3515: MINOR: Make streams quick start more interactive

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3515 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

[GitHub] kafka pull request #3574: HOTFIX: handle commit failed exception on stream t...

2017-07-25 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3574 HOTFIX: handle commit failed exception on stream thread's suspend task 1. Capture `CommitFailedException` in `StreamThread#suspendTasksAndState`; also fix log4j outputs for error and warn.

[jira] [Created] (KAFKA-5639) Enhance DescribeGroups API to return additional group information

2017-07-25 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-5639: -- Summary: Enhance DescribeGroups API to return additional group information Key: KAFKA-5639 URL: https://issues.apache.org/jira/browse/KAFKA-5639 Project: Kafka

[GitHub] kafka pull request #3573: KAFKA-5630; The consumer should block on courrupt ...

2017-07-25 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/3573 KAFKA-5630; The consumer should block on courrupt records and keeping throw exception This patch handles the case that a CorruptRecordException is thrown from the iterator directly. You can

Re: Kafka Connect suggestion before creating new KIP

2017-07-25 Thread Ewen Cheslack-Postava
On Mon, Jul 24, 2017 at 1:50 PM, Florian Hussonnois wrote: > Hi all, > > Here is two suggestions for the Kafka Connect API that I prefer to present > you first before creating KIPs. > > 1/ > Currently, in Kafka Connect we distinguish two configuration types: the > worgker

[jira] [Created] (KAFKA-5641) Metadata request should be allowed to send no regardless of value for max.in.flight.requests.per.connection

2017-07-25 Thread huxihx (JIRA)
huxihx created KAFKA-5641: - Summary: Metadata request should be allowed to send no regardless of value for max.in.flight.requests.per.connection Key: KAFKA-5641 URL: https://issues.apache.org/jira/browse/KAFKA-5641

Build failed in Jenkins: kafka-trunk-jdk7 #2565

2017-07-25 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on cassandra12 (cassandra ubuntu) in workspace

[jira] [Resolved] (KAFKA-5627) Reduce classes needed for LeaderAndIsrPartitionState and MetadataPartitionState

2017-07-25 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5627?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin resolved KAFKA-5627. - Resolution: Fixed > Reduce classes needed for LeaderAndIsrPartitionState and >

Build failed in Jenkins: kafka-trunk-jdk7 #2563

2017-07-25 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on cassandra12 (cassandra ubuntu) in workspace

[GitHub] kafka pull request #3565: KAFKA-5627; Reduce classes needed for LeaderAndIsr...

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3565 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is

Re: [DISCUSS] KIP-174 - Deprecate and remove internal converter configs in WorkerConfig

2017-07-25 Thread Ewen Cheslack-Postava
Umesh, Thanks for the KIP. Straightforward and I think it's a good change. Unfortunately it is hard to tell how many people it would affect since we can't tell how many people have adjusted that config, but I think this is the right thing to do long term. A couple of quick things that might be

Re: consumer group offset lag

2017-07-25 Thread Abhimanyu Nagrath
I am also facing the same issue. On Tue, Jul 25, 2017 at 9:58 PM, Tarun Garg wrote: > I am sending this mail to DEV because i think this is a bug in AdminClient > while getting the group offset. > > Hi, > > I run through a issue. > i am trying to get the status of a consumer

Re: [VOTE] KIP-164 Add unavailablePartitionCount and per-partition Unavailable metrics

2017-07-25 Thread Joel Koshy
+1 On Thu, Jul 20, 2017 at 10:30 AM, Becket Qin wrote: > +1, Thanks for the KIP. > > On Thu, Jul 20, 2017 at 7:08 AM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > On Thu, Jun 1, 2017 at 9:44 AM, Dong Lin

Re: [DISCUSS] KIP-169 Lag-Aware Partition Assignment Strategy

2017-07-25 Thread Ewen Cheslack-Postava
Thanks for the KIP Grant. First, to Vahid's feedback, I think lag is a pretty reasonable heuristic, despite all those other factors. In normal cases I wouldn't expect network latency, per-message processing time, and consumer configuration to vary much within the group. Per-message processing

Build failed in Jenkins: kafka-trunk-jdk7 #2564

2017-07-25 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on cassandra12 (cassandra ubuntu) in workspace

[jira] [Created] (KAFKA-5640) Look into making acks=all the default setting.

2017-07-25 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5640: --- Summary: Look into making acks=all the default setting. Key: KAFKA-5640 URL: https://issues.apache.org/jira/browse/KAFKA-5640 Project: Kafka Issue Type:

[GitHub] kafka pull request #3575: KAFKA-5634; Do not allow segment deletion beyond h...

2017-07-25 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3575 KAFKA-5634; Do not allow segment deletion beyond high watermark You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-5634