Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Ewen Cheslack-Postava
I have the same initial response as Ismael re: broker vs consumer settings. The global setting seems questionable. Could we maybe summarize what the impact of making this a client config would be? Protocol bump is obvious, but is there any other significant issue? For the protocol bump in

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-23 Thread Jun Rao
Hi, Dong, 11.2 I think there are a few reasons why the cross disk movement may not catch up if the replicas are created in the wrong log dirs to start with. (a) There could be more replica fetcher threads than the disk movement threads. (b) intra.broker.throttled.rate may be configured lower than

Re: [VOTE] KIP-82 Add Record Headers

2017-03-23 Thread Jun Rao
Hi, Ismael, Ok, that make sense. Hi, Michael, Could we document this in the wiki? +1 from me on the KIP. Thanks, Jun On Thu, Mar 23, 2017 at 7:58 PM, Ismael Juma wrote: > Hi Jun, > > The close method will only exist in the implementation class as it's not > meant to be

Re: [VOTE] KIP-123: Allow per stream/table timestamp extractor

2017-03-23 Thread Matthias J. Sax
Thanks Jeyhun. Can you also update the KIP accordingly. It must contain all changes to public API. Thus, list all parameters that get deprecated and newly added. And add a sentence about backward compatibility. -Matthias On 3/23/17 3:16 AM, Jeyhun Karimov wrote: > Sorry for a super late

Re: [VOTE] KIP-82 Add Record Headers

2017-03-23 Thread Ismael Juma
Hi Jun, The close method will only exist in the implementation class as it's not meant to be called from user code. Ismael On 24 Mar 2017 1:51 am, "Jun Rao" wrote: > Hi, Michael, > > The KIP looks good to me overall. Just one comment. The wiki says "This > will be done by

Re: [VOTE] KIP-82 Add Record Headers

2017-03-23 Thread Jun Rao
Hi, Michael, The KIP looks good to me overall. Just one comment. The wiki says "This will be done by calling "close()" method". However, there is no close() in Headers. Thanks, Jun On Thu, Mar 23, 2017 at 9:34 AM, Michael Pearce wrote: > Thanks all for the +1 votes so

[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected

2017-03-23 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15939464#comment-15939464 ] ASF GitHub Bot commented on KAFKA-4942: --- GitHub user simplesteph opened a pull request:

[GitHub] kafka pull request #2730: KAFKA-4942 fixed the commitTimeoutMs being set bef...

2017-03-23 Thread simplesteph
GitHub user simplesteph opened a pull request: https://github.com/apache/kafka/pull/2730 KAFKA-4942 fixed the commitTimeoutMs being set before the commit actually started this fixes KAFKA-4942 You can merge this pull request into a Git repository by running: $ git pull

[jira] [Comment Edited] (KAFKA-4942) Kafka Connect: Offset committing times out before expected

2017-03-23 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15939453#comment-15939453 ] Stephane Maarek edited comment on KAFKA-4942 at 3/23/17 11:57 PM: --

[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected

2017-03-23 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15939453#comment-15939453 ] Stephane Maarek commented on KAFKA-4942: [~ewencp] nothing weird from the log, except that EVERY

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Matthias J. Sax
Thanks for the KIP Damian! My two cents: - we should have an explicit parameter for this -- implicit setting are always tricky (the "importance" of this parameter would be LOW) - the config should be different for each consumer group: * assume you have a stateless app, you want to

Build failed in Jenkins: kafka-0.10.2-jdk7 #113

2017-03-23 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: only log first exception in RecordCollectorImpl producer callback -- [...truncated 754.59 KB...]

[VOTE] KIP-120: Cleanup Kafka Streams builder API

2017-03-23 Thread Matthias J. Sax
Hi, I would like to start the VOTE on KIP-120: https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API If you have further comments, please reply to the DISCUSS thread. Thanks a lot! -Matthias signature.asc Description: OpenPGP digital signature

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-23 Thread Matthias J. Sax
Jay, about the naming schema: >>1. "kstreams" - the DSL >>2. "processor api" - the lower level callback/topology api >>3. KStream/KTable - entities in the kstreams dsl >>4. "Kafka Streams" - General name for stream processing stuff in Kafka, >>including both kstreams and the

[jira] [Work started] (KAFKA-4916) Add streams tests with brokers failing

2017-03-23 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4916 started by Eno Thereska. --- > Add streams tests with brokers failing > --

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

2017-03-23 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: only log first exception in RecordCollectorImpl producer callback -- [...truncated 165.14 KB...]

Jenkins build is back to normal : kafka-0.10.2-jdk7 #112

2017-03-23 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-trunk-jdk8 #1373

2017-03-23 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-03-23 Thread Jorge Esteban Quilcate Otoya
@Ismael, thanks for your feedback! 1. Good point. I will add optional support for timezone as part of the datetime input. But, when datetime is without timezone, would it be more consistent to get the timezone from the cluster first and then reset based on that value? Not sure if it is possible to

[jira] [Updated] (KAFKA-4810) SchemaBuilder should be more lax about checking that fields are unset if they are being set to the same value

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4810: - Labels: newbie (was: ) > SchemaBuilder should be more lax about checking that

[jira] [Updated] (KAFKA-4280) Document REST resource for showing available connector plugin configs

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4280: - Summary: Document REST resource for showing available connector plugin configs

[jira] [Updated] (KAFKA-4855) Struct SchemaBuilder should not allow duplicate fields.

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4855?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4855: - Labels: newbie (was: ) > Struct SchemaBuilder should not allow duplicate fields.

[jira] [Updated] (KAFKA-4400) Prefix for sink task consumer groups should be configurable

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4400?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4400: - Labels: newbie (was: ) > Prefix for sink task consumer groups should be

[jira] [Commented] (KAFKA-4942) Kafka Connect: Offset committing times out before expected

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938892#comment-15938892 ] Ewen Cheslack-Postava commented on KAFKA-4942: -- [~stephane.maa...@gmail.com] Any more context

[jira] [Updated] (KAFKA-4278) Undocumented REST resources

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4278: - Labels: newbie (was: ) > Undocumented REST resources >

[jira] [Updated] (KAFKA-4938) Creating a connector with missing name parameter throws a NullPointerException

2017-03-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4938: - Labels: newbie (was: ) > Creating a connector with missing name parameter throws

[jira] [Commented] (KAFKA-4945) Suppress findbugs warnings about machine-generated code in jmh-benchmarks

2017-03-23 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938817#comment-15938817 ] ASF GitHub Bot commented on KAFKA-4945: --- GitHub user cmccabe opened a pull request:

[GitHub] kafka pull request #2729: KAFKA-4945: Suppress findbugs warnings about machi...

2017-03-23 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2729 KAFKA-4945: Suppress findbugs warnings about machine-generated code i… …n jmh-benchmarks You can merge this pull request into a Git repository by running: $ git pull

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Damian Guy
Thanks Gouzhang - i think another problem with this is that is overloading session.timeout.ms to mean multiple things. I'm not sure that is a good thing. On Thu, 23 Mar 2017 at 17:14 Guozhang Wang wrote: > The downside of it, though, is that although it "hides" this from

[jira] [Created] (KAFKA-4945) Suppress findbugs warnings about machine-generated code in jmh-benchmarks

2017-03-23 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-4945: -- Summary: Suppress findbugs warnings about machine-generated code in jmh-benchmarks Key: KAFKA-4945 URL: https://issues.apache.org/jira/browse/KAFKA-4945 Project:

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Guozhang Wang
The downside of it, though, is that although it "hides" this from most of the users needing to be aware of it, by default session timeout i.e. the rebalance timeout is 10 seconds which could arguably too long. Guozhang On Thu, Mar 23, 2017 at 10:12 AM, Guozhang Wang wrote:

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Guozhang Wang
Just throwing another alternative idea here: we can consider using the rebalance timeout value which is already included in the join request protocol (and on the current Java client it is always written as the session timeout value), that the first member joining will always force the coordinator

[GitHub] kafka pull request #2726: MINOR: only log first exception in RecordCollector...

2017-03-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2726 --- 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

[jira] [Commented] (KAFKA-4392) Failed to lock the state directory due to an unexpected exception

2017-03-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4392?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938752#comment-15938752 ] Guozhang Wang commented on KAFKA-4392: -- [~elevy] Could you share your settings (num.threads per

[jira] [Resolved] (KAFKA-4919) Document that stores must not be closed when Processors are closed

2017-03-23 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4919. -- Resolution: Fixed Fix Version/s: 0.10.2.1 0.11.0.0 > Document that

[jira] [Commented] (KAFKA-4919) Document that stores must not be closed when Processors are closed

2017-03-23 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4919?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938749#comment-15938749 ] ASF GitHub Bot commented on KAFKA-4919: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2725: KAFKA-4919: Document that stores must not be close...

2017-03-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2725 --- 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 #2728: KAFKA 3514: alternate calculation for min timestam...

2017-03-23 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2728 KAFKA 3514: alternate calculation for min timestamp [WiP] This alternate calculation reduces the amount of buffering kept and is optimised for cases when out of order records are rare. You

[jira] [Commented] (KAFKA-4943) SCRAM secret's should be better protected with Zookeeper ACLs

2017-03-23 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938726#comment-15938726 ] Rajini Sivaram commented on KAFKA-4943: --- The current SCRAM implementation uses the default ACL

Re: [VOTE] KIP-82 Add Record Headers

2017-03-23 Thread Michael Pearce
Thanks all for the +1 votes so far, just one more committer vote needed ( Please note: I have made one small adjustment to the kip based on Ismael’s comment in discussion thread, and further requested by Jason in the vote thread. Please note the below method is changed based on this feedback.

[jira] [Commented] (KAFKA-4159) Allow overriding producer & consumer properties at the connector level

2017-03-23 Thread Stephen Durfey (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938692#comment-15938692 ] Stephen Durfey commented on KAFKA-4159: --- So, full disclosure, I believe my original desire for

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-03-23 Thread Ismael Juma
Hi Jorge, Thanks for the detailed KIP. The tool looks very useful. A few comments: 1. We are using the default timezone of the client for the specified date. This seems a bit error prone. Would it be better to require the users to specify the time zone as part of the date time? We should at

Kafka queue full configuration

2017-03-23 Thread Mohapatra, Sudhir (Nokia - IN/Gurgaon)
Hi, We are trying to simulate the kafka queue full scenarios on kafka 0.10.0. I have seen in earlier versions there is a configuration parameter "queue.buffering.max.messages" which can be set to simulate the queue full scenario. But in the kafka 0.10.0 this parameter is not there.

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-03-23 Thread Mickael Maison
+1 (non binding) Thanks for the KIP On Wed, Mar 22, 2017 at 11:00 AM, Jorge Esteban Quilcate Otoya wrote: > @Jason, thanks for your feedback! > You're right, we are not considering the old consumer, since we rely on the > KafkaConsumer#seek operations. I'm happy to

[jira] [Commented] (KAFKA-4944) Fix an "unread field" findbugs warning in streams examples

2017-03-23 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938589#comment-15938589 ] ASF GitHub Bot commented on KAFKA-4944: --- GitHub user cmccabe opened a pull request:

[GitHub] kafka pull request #2727: KAFKA-4944. Fix an "unread field" findbugs warning...

2017-03-23 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2727 KAFKA-4944. Fix an "unread field" findbugs warning in streams examples You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka

[jira] [Created] (KAFKA-4944) Fix an "unread field" findbugs warning in streams examples

2017-03-23 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-4944: -- Summary: Fix an "unread field" findbugs warning in streams examples Key: KAFKA-4944 URL: https://issues.apache.org/jira/browse/KAFKA-4944 Project: Kafka

Build failed in Jenkins: kafka-trunk-jdk8 #1372

2017-03-23 Thread Apache Jenkins Server
See -- [...truncated 710.12 KB...] org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreTest > testRestore STARTED

[jira] [Commented] (KAFKA-4930) Connect Rest API allows creating connectors with an empty name

2017-03-23 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938458#comment-15938458 ] Sönke Liebau commented on KAFKA-4930: - After a good nights sleep I am beginning to think that between

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Damian Guy
Hi Ismael, Mostly to avoid the protocol bump. I agree that it may be difficult to choose the right delay for all consumer groups, but we wanted to make this something that most users don't really need to think about, i.e., a small enough default delay that works in the majority of cases. However

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Ismael Juma
Thanks for the KIP, Damian. It makes sense to avoid multiple rebalances during start-up. One issue with having this as a broker config is that it may be difficult to choose the right delay for all consumer groups. Can you elaborate a little more on why the first alternative (add a consumer config)

[DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-23 Thread Damian Guy
Hi All, I've prepared a KIP to add a configurable delay to the initial consumer group rebalance. Please have look here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-134%3A+Delay+initial+consumer+group+rebalance Thanks, Damian BTW, i apologize if this appears twice. Seems the first one

[jira] [Assigned] (KAFKA-4919) Document that stores must not be closed when Processors are closed

2017-03-23 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-4919: - Assignee: Damian Guy > Document that stores must not be closed when Processors are closed >

[jira] [Work started] (KAFKA-4919) Document that stores must not be closed when Processors are closed

2017-03-23 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4919 started by Damian Guy. - > Document that stores must not be closed when Processors are closed >

[GitHub] kafka pull request #2726: MINOR: only log first exception in RecordCollector...

2017-03-23 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2726 MINOR: only log first exception in RecordCollectorImpl producer callback You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka

[jira] [Commented] (KAFKA-4943) SCRAM secret's should be better protected with Zookeeper ACLs

2017-03-23 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15938060#comment-15938060 ] Ismael Juma commented on KAFKA-4943: cc [~rsivaram] > SCRAM secret's should be better protected with

Re: [VOTE] KIP-123: Allow per stream/table timestamp extractor

2017-03-23 Thread Jeyhun Karimov
Sorry for a super late update. I made an update on related PR. Cheers, Jeyhun On Wed, Mar 22, 2017 at 9:09 PM Guozhang Wang wrote: > Jeyhun, > > Could you update the status of this KIP since it has been some time since > the last vote? > > I'm +1 besides the minor comments

[jira] [Commented] (KAFKA-4943) SCRAM secret's should be better protected with Zookeeper ACLs

2017-03-23 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15937993#comment-15937993 ] Johan Ström commented on KAFKA-4943: Seems the no-acl-set-at-all was fixed in

[jira] [Created] (KAFKA-4943) SCRAM secret's should be better protected with Zookeeper ACLs

2017-03-23 Thread JIRA
Johan Ström created KAFKA-4943: -- Summary: SCRAM secret's should be better protected with Zookeeper ACLs Key: KAFKA-4943 URL: https://issues.apache.org/jira/browse/KAFKA-4943 Project: Kafka

Jenkins build is back to normal : kafka-trunk-jdk7 #2036

2017-03-23 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-23 Thread Ismael Juma
Thanks for the KIP Mayuresh. I suggest we only throw this error for ProduceRequest version 3, which is being introduced with KIP-98 (Exactly-once). That way, the compatibility story is clearer, in my opinion. Ismael On Wed, Mar 22, 2017 at 10:07 PM, Mayuresh Gharat < gharatmayures...@gmail.com>

Build failed in Jenkins: kafka-0.10.2-jdk7 #111

2017-03-23 Thread Apache Jenkins Server
See Changes: [me] MINOR: Adding example to SMT documentation -- [...truncated 160.40 KB...] kafka.server.KafkaConfigTest > testUncleanLeaderElectionDefault STARTED

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-23 Thread Mayuresh Gharat
Hi James, I meant that "it is recommended to upgrade clients before upgrading the brokers". Will update the KIP to reflect that. Thanks, Mayuresh On Wed, Mar 22, 2017 at 4:42 PM, James Cheng wrote: > Mayuresh, > > The Compatibility/Migration section says to upgrade the

[GitHub] kafka pull request #2721: Minor: Adding example to SMT documentation

2017-03-23 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2721 --- 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