Re: [DISCUSS] 0.11.0.1 bug fix release

2017-08-16 Thread Srikanth Sampath
Thanks Damian. What's the ballpark when 0.11.0.1 will be available? -Srikanth On Wed, Aug 16, 2017 at 5:59 PM, Damian Guy wrote: > Hi, > > It seems like it must be time for 0.11.0.1 bug fix release! > > Since the 0.11.0.0 release we've fixed 30 JIRAs that > are targeted for 0.11.0.1: > > https:

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-16 Thread Apurva Mehta
Thanks for the followup Becket. It sounds we are on agreement on the scope of this KIP, and the discussion has definitely clarified a lot of the subtle points. Apurva On Tue, Aug 15, 2017 at 10:49 PM, Becket Qin wrote: > Hi Apurva, > > Thanks for the clarification of the definition. The definit

[GitHub] kafka pull request #3682: KAFKA-5745: makeLeader should invoke `convertHWToL...

2017-08-16 Thread huxihx
GitHub user huxihx opened a pull request: https://github.com/apache/kafka/pull/3682 KAFKA-5745: makeLeader should invoke `convertHWToLocalOffsetMetadata` before marking it as leader You can merge this pull request into a Git repository by running: $ git pull https://github.co

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-16 Thread Jun Rao
Hi, Rajini, Thanks for the KIP. A few comments. 1. We have 30+ requests and 30+ error code and growing. So, the combination can be large. Perhaps it's useful to expire an error metric if it's no longer updated after some time? We did something similar for the quota metric. 2. It's a bit weird to

[jira] [Created] (KAFKA-5745) Partition.makeLeader() should convert HW to OffsetMetadata before becoming the leader

2017-08-16 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-5745: -- Summary: Partition.makeLeader() should convert HW to OffsetMetadata before becoming the leader Key: KAFKA-5745 URL: https://issues.apache.org/jira/browse/KAFKA-5745 Project: Kafk

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

2017-08-16 Thread Apache Jenkins Server
See -- [...truncated 914.17 KB...] kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetT

[DISCUSS] KIP-189: Improve principal builder interface and add support for SASL

2017-08-16 Thread Jason Gustafson
Hi All, I've added a new KIP to improve and extend the principal building API that Kafka exposes: https://cwiki.apache.org/confluence/display/KAFKA/KIP-189%3A+Improve+principal+builder+interface+and+add+support+for+SASL . As always, feedback is appreciated. Thanks, Jason

[GitHub] kafka pull request #3681: KAFKA-5733: RocksDB bulk load with lower number of...

2017-08-16 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3681 KAFKA-5733: RocksDB bulk load with lower number of levels. This is to complete Bill's PR on KAFKA-5733, incorporating the suggestion in https://github.com/facebook/rocksdb/issues/2734.

[GitHub] kafka pull request #3680: KAFKA-5743. Ducktape services should use subdirs o...

2017-08-16 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/3680 KAFKA-5743. Ducktape services should use subdirs of /mnt You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-5743 Alternatively

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

2017-08-16 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-187 - Add cumulative count metric for all Kafka rate metrics

2017-08-16 Thread Roger Hoover
Rajini, Thank you. This is very useful. Grouping by metric by prefixing the name instead of making them MBeans is not quite as nice but seems like an good compromise for backward compatibility. Cheers, Roger On Wed, Aug 16, 2017 at 5:35 AM, Rajini Sivaram wrote: > Sorry, pressed send by mis

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-16 Thread Roger Hoover
I think it would useful to make clear somewhere for each metric, the level at which it's counted. I don't know all the details of the Kafka protocol but it might be something like ProduceRequest, Fetch Request - counted at per-partition level All other requests are 1:1 with client requests? Chee

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-16 Thread Roger Hoover
Rajini, Thank you for the KIP. These are very helpful additions. One question on the error code metrics: Will the total error counting happen at the the level of topic partition? For example, if a single ProduceRequest contains messages to append to 3 partitions and say all 3 appends are succes

[GitHub] kafka pull request #3679: KAFKA-5744: ShellTest: add tests for attempting to...

2017-08-16 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/3679 KAFKA-5744: ShellTest: add tests for attempting to run nonexistent pr… …ogram, error return You can merge this pull request into a Git repository by running: $ git pull https://github.com/c

[jira] [Created] (KAFKA-5744) ShellTest: add tests for attempting to run nonexistent program, error return

2017-08-16 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5744: -- Summary: ShellTest: add tests for attempting to run nonexistent program, error return Key: KAFKA-5744 URL: https://issues.apache.org/jira/browse/KAFKA-5744 Projec

[GitHub] kafka pull request #2862: KAFKA-5076: remove usage of java.xml.bind.* classe...

2017-08-16 Thread xvrl
Github user xvrl closed the pull request at: https://github.com/apache/kafka/pull/2862 --- 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 enable

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

2017-08-16 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-5743) Ducktape services should store their files in subdirectories of /mnt

2017-08-16 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5743: -- Summary: Ducktape services should store their files in subdirectories of /mnt Key: KAFKA-5743 URL: https://issues.apache.org/jira/browse/KAFKA-5743 Project: Kafka

[GitHub] kafka pull request #3678: KAFKA-5233 follow up

2017-08-16 Thread mihbor
GitHub user mihbor opened a pull request: https://github.com/apache/kafka/pull/3678 KAFKA-5233 follow up You can merge this pull request into a Git repository by running: $ git pull https://github.com/mihbor/kafka trunk Alternatively you can review and apply these changes as

[jira] [Resolved] (KAFKA-5567) With transformations that mutate the topic-partition committing offsets should to refer to the original topic-partition

2017-08-16 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5567. -- Resolution: Fixed > With transformations that mutate the topic-partition committ

[GitHub] kafka pull request #3499: KAFKA-5567: Connect sink worker should commit offs...

2017-08-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3499 --- 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 enab

[GitHub] kafka pull request #3677: KAFKA-5742 support ZK chroot in system tests

2017-08-16 Thread xvrl
GitHub user xvrl opened a pull request: https://github.com/apache/kafka/pull/3677 KAFKA-5742 support ZK chroot in system tests You can merge this pull request into a Git repository by running: $ git pull https://github.com/xvrl/kafka support-zk-chroot-in-tests Alternatively y

[jira] [Created] (KAFKA-5742) Support passing ZK chroot in system tests

2017-08-16 Thread JIRA
Xavier Léauté created KAFKA-5742: Summary: Support passing ZK chroot in system tests Key: KAFKA-5742 URL: https://issues.apache.org/jira/browse/KAFKA-5742 Project: Kafka Issue Type: Test

[GitHub] kafka pull request #3676: MINOR: Ensure consumer logging has clientId/groupI...

2017-08-16 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3676 MINOR: Ensure consumer logging has clientId/groupId context You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka log-consumer-wakeu

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-16 Thread Jason Gustafson
+1 On Sun, Aug 13, 2017 at 5:15 PM, Guozhang Wang wrote: > +1. Thanks. > > On Sat, Aug 12, 2017 at 11:18 AM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). Seems like the KIP title has to be > > updated still. > > > > Ismael > > > > On Fri, Aug 11, 2017 at 11:00 PM, Colin McCabe >

[DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-16 Thread Rajini Sivaram
I have created a KIP to add some additional metrics to support health checks: https://cwiki.apache.org/confluence/display/KAFKA/KIP-188+-+Add+new+metrics+to+support+health+checks Feedback and suggestions are welcome. Regards, Rajini

Re: [DISCUSS] 0.11.0.1 bug fix release

2017-08-16 Thread Guozhang Wang
Thanks Damian for driving the release! On Wed, Aug 16, 2017 at 5:40 AM, Ismael Juma wrote: > Thanks for driving the release Damian. Sounds good to me. > > Ismael > > On Wed, Aug 16, 2017 at 1:29 PM, Damian Guy wrote: > > > Hi, > > > > It seems like it must be time for 0.11.0.1 bug fix release!

[jira] [Resolved] (KAFKA-3796) SslTransportLayerTest.testInvalidEndpointIdentification fails on trunk

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3796?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3796. -- Resolution: Cannot Reproduce Pl reopen if you think the issue still exists > SslTransportLayerTest.tes

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-08-16 Thread Jun Rao
Hi, Sumant, The semantics of linger.ms is a bit subtle. The reasoning for the current implementation is the following. Let's say one sets linger.ms to 0 (our current default value). Creating a batch for every message will be bad for throughput. Instead, the current implementation only forms a batc

[jira] [Resolved] (KAFKA-4803) OT6Y1

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4803. -- Resolution: Invalid > OT6Y1 > - > > Key: KAFKA-4803 > URL: https://i

[jira] [Resolved] (KAFKA-4813) 2h6R1

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4813?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4813. -- Resolution: Invalid > 2h6R1 > - > > Key: KAFKA-4813 > URL: https://i

[jira] [Resolved] (KAFKA-4804) TdOZY

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4804. -- Resolution: Invalid > TdOZY > - > > Key: KAFKA-4804 > URL: https://i

[jira] [Resolved] (KAFKA-4821) 9244L

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4821. -- Resolution: Invalid > 9244L > - > > Key: KAFKA-4821 > URL: https://i

[jira] [Resolved] (KAFKA-4865) 2X8BF

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4865. -- Resolution: Invalid > 2X8BF > - > > Key: KAFKA-4865 > URL: https://i

[jira] [Created] (KAFKA-5741) Prioritize threads in Connect distributed worker process

2017-08-16 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5741: Summary: Prioritize threads in Connect distributed worker process Key: KAFKA-5741 URL: https://issues.apache.org/jira/browse/KAFKA-5741 Project: Kafka Issue

[jira] [Resolved] (KAFKA-4847) 1Y30J

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4847. -- Resolution: Invalid > 1Y30J > - > > Key: KAFKA-4847 > URL: https://i

[jira] [Resolved] (KAFKA-4889) 2G8lc

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4889?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4889. -- Resolution: Invalid > 2G8lc > - > > Key: KAFKA-4889 > URL: https://i

[jira] [Resolved] (KAFKA-4951) KafkaProducer may send duplicated message sometimes

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4951?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4951. -- Resolution: Fixed This scenario is handled in the Idempotent producer (KIP-98) released in Kafka 0.11.0

[jira] [Created] (KAFKA-5740) Use separate file for HTTP logs

2017-08-16 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5740: Summary: Use separate file for HTTP logs Key: KAFKA-5740 URL: https://issues.apache.org/jira/browse/KAFKA-5740 Project: Kafka Issue Type: Improvement

Re: KAFKA-5726

2017-08-16 Thread Jun Rao
Hi, Attila, Thanks for your interest. I just added you to the contributor list. Jun On Wed, Aug 16, 2017 at 1:07 AM, Attila Kreiner wrote: > Hi All, > > I am new here, so I picked an easy issue and created a PR: > https://github.com/apache/kafka/pull/3669 > https://issues.apache.org/jira/brows

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-16 Thread Ismael Juma
It needs a KIP if it introduces new public APIs and/or there are compatibility implications. Otherwise, no KIP is required. Ismael On Wed, Aug 16, 2017 at 4:38 PM, Paolo Patierno wrote: > Hi Ismael, > > > after your first review on my PR and the conversation here in the mailing > list ... I hav

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-16 Thread Paolo Patierno
Hi Ismael, after your first review on my PR and the conversation here in the mailing list ... I have a doubt ... We are going to remove the --zookeeper option but adding the --broker-list but according to this discussion we agree on your solution having the choice at shell script level. Does

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

2017-08-16 Thread Apache Jenkins Server
See -- [...truncated 914.27 KB...] kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetT

KAFKA-5726

2017-08-16 Thread Attila Kreiner
Hi All, I am new here, so I picked an easy issue and created a PR: https://github.com/apache/kafka/pull/3669 https://issues.apache.org/jira/browse/KAFKA-5726 Can someone please review it? I also wish to be added to the contributor list so I can update the issue in Jira. Thanks, Attila

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

2017-08-16 Thread Apache Jenkins Server
See

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

2017-08-16 Thread Apache Jenkins Server
See -- [...truncated 915.58 KB...] kafka.controller.ControllerFailoverTest > testHandleIllegalStateException STARTED kafka.controller.ControllerFailoverTest > testHandleIllegalStateExc

[GitHub] kafka pull request #3675: KAFKA-5152: perform state restoration in poll loop

2017-08-16 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3675 KAFKA-5152: perform state restoration in poll loop You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka kafka-5152 Alternatively you can rev

[jira] [Created] (KAFKA-5739) Rewrite KStreamPeekTest at processor level avoiding driver usage

2017-08-16 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5739: - Summary: Rewrite KStreamPeekTest at processor level avoiding driver usage Key: KAFKA-5739 URL: https://issues.apache.org/jira/browse/KAFKA-5739 Project: Kafka

[GitHub] kafka pull request #3170: KAFKA-5054: ChangeLoggingKeyValueByteStore delete ...

2017-08-16 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/3170 --- 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 enable

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

2017-08-16 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5737; KafkaAdminClient thread should be daemon -- [...truncated 913.21 KB...] kafka.server.ServerStartupTest > testBrokerCreatesZKChroot START

[jira] [Resolved] (KAFKA-5611) One or more consumers in a consumer-group stop consuming after rebalancing

2017-08-16 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5611. Resolution: Fixed I'm going to mark this as closed. If [~pskianis] or someone else can still see th

Re: [DISCUSS] 0.11.0.1 bug fix release

2017-08-16 Thread Ismael Juma
Thanks for driving the release Damian. Sounds good to me. Ismael On Wed, Aug 16, 2017 at 1:29 PM, Damian Guy wrote: > Hi, > > It seems like it must be time for 0.11.0.1 bug fix release! > > Since the 0.11.0.0 release we've fixed 30 JIRAs that > are targeted for 0.11.0.1: > > https://issues.apac

Re: [DISCUSS] KIP-187 - Add cumulative count metric for all Kafka rate metrics

2017-08-16 Thread Rajini Sivaram
Sorry, pressed send by mistake. The KIP is here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-187+-+Add+cumulative+count+metric+for+all+Kafka+rate+metrics Feedback and suggestions are appreciated. Thank you... Rajini On Wed, Aug 16, 2017 at 1:34 PM, Rajini Sivaram wrote: > Hi all,

[DISCUSS] KIP-187 - Add cumulative count metric for all Kafka rate metrics

2017-08-16 Thread Rajini Sivaram
Hi all, I have posted a KIP to add a cumulative count attribute for all Kafka rate metrics to make downstream processing simpler, more accurate and flexible:

[jira] [Resolved] (KAFKA-3087) Fix documentation for retention.ms property and update documentation for LogConfig.scala class

2017-08-16 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3087?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3087. -- Resolution: Fixed This doc issue was fixed in newer Kafka versions. > Fix documentation for retention.m

[DISCUSS] 0.11.0.1 bug fix release

2017-08-16 Thread Damian Guy
Hi, It seems like it must be time for 0.11.0.1 bug fix release! Since the 0.11.0.0 release we've fixed 30 JIRAs that are targeted for 0.11.0.1: https://issues.apache.org/jira/browse/KAFKA-5700?jql=project%20%3D%20KAFKA%20AND%20resolution%20%3D%20Fixed%20AND%20fixVersion%20%3D%200.11.0.1%20ORDER%

[GitHub] kafka pull request #3674: KAFKA-5737. KafkaAdminClient thread should be daem...

2017-08-16 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3674 --- 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 enab

[jira] [Created] (KAFKA-5738) Add cumulative count attribute for all Kafka rate metrics

2017-08-16 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-5738: - Summary: Add cumulative count attribute for all Kafka rate metrics Key: KAFKA-5738 URL: https://issues.apache.org/jira/browse/KAFKA-5738 Project: Kafka Iss

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2017-08-16 Thread Stevo Slavić
+1 for making consistent default log and offsets retention time. I like Stephane's suggestion too, log retention override should override offset retention too if not explicitly configured. Please consider additionally: - introducing offsets.retention.hours config property - syncing log and offsets

[GitHub] kafka pull request #3653: KAFKA-5152: move state restoration out of rebalanc...

2017-08-16 Thread dguy
Github user dguy closed the pull request at: https://github.com/apache/kafka/pull/3653 --- 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 enable

[jira] [Resolved] (KAFKA-5440) Kafka Streams report state RUNNING even if all threads are dead

2017-08-16 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-5440. --- Resolution: Duplicate > Kafka Streams report state RUNNING even if all threads are dead >

[jira] [Resolved] (KAFKA-5684) KStreamPrintProcessor as customized KStreamPeekProcessor

2017-08-16 Thread Paolo Patierno (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paolo Patierno resolved KAFKA-5684. --- Resolution: Feedback Received > KStreamPrintProcessor as customized KStreamPeekProcessor > ---

[GitHub] kafka pull request #3636: KAFKA-5684: KStreamPrintProcessor as customized KS...

2017-08-16 Thread ppatierno
Github user ppatierno closed the pull request at: https://github.com/apache/kafka/pull/3636 --- 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 e