Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread Daniel Schierbeck
I don't have a lot of feedback on this, but at Zendesk we could definitely use a standardized header system. Using ints as keys sounds tedious, but if that's a necessary tradeoff I'd be okay with it. On Fri, Dec 2, 2016 at 5:44 AM Todd Palino wrote: > Come on, I’ve done at least 2 talks on this

[GitHub] kafka pull request #1548: KAFKA-2857 ConsumerGroupCommand throws GroupCoordi...

2016-12-02 Thread imandhan
Github user imandhan closed the pull request at: https://github.com/apache/kafka/pull/1548 --- 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 en

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714543#comment-15714543 ] ASF GitHub Bot commented on KAFKA-2857: --- Github user imandhan closed the pull reques

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714623#comment-15714623 ] ASF GitHub Bot commented on KAFKA-2857: --- GitHub user imandhan opened a pull request:

[GitHub] kafka pull request #2203: KAFKA-2857 ConsumerGroupCommand throws GroupCoordi...

2016-12-02 Thread imandhan
GitHub user imandhan opened a pull request: https://github.com/apache/kafka/pull/2203 KAFKA-2857 ConsumerGroupCommand throws GroupCoordinatorNotAvailableEx… …ception when describing a non-existent group before the offset topic is created #1548 You can merge this pull request i

[jira] [Commented] (KAFKA-4474) Poor kafka-streams throughput

2016-12-02 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714644#comment-15714644 ] Juan Chorro commented on KAFKA-4474: [~enothereska] thanks for you quick response. Th

[jira] [Comment Edited] (KAFKA-4474) Poor kafka-streams throughput

2016-12-02 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714644#comment-15714644 ] Juan Chorro edited comment on KAFKA-4474 at 12/2/16 9:49 AM: -

[jira] [Comment Edited] (KAFKA-4474) Poor kafka-streams throughput

2016-12-02 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714644#comment-15714644 ] Juan Chorro edited comment on KAFKA-4474 at 12/2/16 9:51 AM: -

[jira] [Updated] (KAFKA-4474) Poor kafka-streams throughput

2016-12-02 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Juan Chorro updated KAFKA-4474: --- Attachment: hctop sreenshot.png Screenshot of htop after 1 minute application running > Poor kafka-st

[jira] [Comment Edited] (KAFKA-4474) Poor kafka-streams throughput

2016-12-02 Thread Juan Chorro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714661#comment-15714661 ] Juan Chorro edited comment on KAFKA-4474 at 12/2/16 9:59 AM: -

[jira] [Comment Edited] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714715#comment-15714715 ] Ismael Juma edited comment on KAFKA-4454 at 12/2/16 10:21 AM: --

[jira] [Commented] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714715#comment-15714715 ] Ismael Juma commented on KAFKA-4454: Yes, I understand that we can simply add a field

[jira] [Created] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-02 Thread Michael Andre Pearce (IG) (JIRA)
Michael Andre Pearce (IG) created KAFKA-4477: Summary: Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted. Key: KAFKA-4477

[jira] [Updated] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4477: --- Labels: reliability (was: ) > Node reduces its ISR to itself, and doesn't recover. Other nodes do not

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-02 Thread Michael Pearce
Hi Jao Thanks for the response. Sorry for slow reply, both with personal sickness and also battling some critical issues encountered since upgrading to 0.10.1.0 1) Thans for spotting, Document error where we branched this KIP from KIP-82, will get that removed. 2) Intent is to do this just at t

[jira] [Commented] (KAFKA-1641) Log cleaner exits if last cleaned offset is lower than earliest offset

2016-12-02 Thread yuanjiali (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1641?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15714892#comment-15714892 ] yuanjiali commented on KAFKA-1641: -- We are hitting this problem in 0.10.0.0. Log as blow:

Re: [DISCUSS] KIP-81: Max in-flight fetches

2016-12-02 Thread Mickael Maison
It's been a few days since the last comments. KIP-72 vote seems to have passed so if I don't get any new comments I'll start the vote on Monday. Thanks On Mon, Nov 14, 2016 at 6:25 PM, radai wrote: > +1 - there's is a need for an effective way to control kafka memory > consumption - both on the b

[jira] [Created] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
Sinóros-Szabó Péter created KAFKA-4478: -- Summary: Deadlock between heartbeat executor, group metadata manager and request handler Key: KAFKA-4478 URL: https://issues.apache.org/jira/browse/KAFKA-4478

[jira] [Updated] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sinóros-Szabó Péter updated KAFKA-4478: --- Affects Version/s: 0.10.1.0 > Deadlock between heartbeat executor, group metadata mana

[jira] [Updated] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sinóros-Szabó Péter updated KAFKA-4478: --- Labels: reliability (was: ) > Deadlock between heartbeat executor, group metadata man

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715077#comment-15715077 ] Ismael Juma commented on KAFKA-4477: Could this be KAFKA-4478? > Node reduces its ISR

[jira] [Updated] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sinóros-Szabó Péter updated KAFKA-4478: --- Attachment: kafka-1.state-change.log kafka-1.server.log.2016-12-01-21

[jira] [Commented] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715082#comment-15715082 ] Ismael Juma commented on KAFKA-4478: Thanks for the report. Could be related to KAFKA-

[jira] [Updated] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sinóros-Szabó Péter updated KAFKA-4478: --- Description: We are running a 0.10.1.0 cluster with 3 brokers with ids 0, 1 and 2. At

[jira] [Updated] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sinóros-Szabó Péter updated KAFKA-4478: --- Description: We are running a 0.10.1.0 cluster with 3 brokers with ids 0, 1 and 2. At

[jira] [Commented] (KAFKA-4478) Deadlock between heartbeat executor, group metadata manager and request handler

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715094#comment-15715094 ] Sinóros-Szabó Péter commented on KAFKA-4478: Yes, locked threads and stack tra

Re: [DISCUSS] KIP-96 - Add per partition metrics for in-sync and assigned replica count

2016-12-02 Thread Ismael Juma
Hi Xavier, Can you please share the reasoning for the name change? Good to record such things for posterity. :) Ismael On Wed, Nov 30, 2016 at 9:20 PM, Xavier Léauté wrote: > FYI, Based on internal feedback I renamed AssignedReplicasCount to simply > be called ReplicasCount. > > On Tue, Nov 29

[jira] [Updated] (KAFKA-4472) offsetRetentionMs miscalculated in GroupCoordinator

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4472?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4472: --- Fix Version/s: 0.10.1.1 > offsetRetentionMs miscalculated in GroupCoordinator > --

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-02 Thread Michael Pearce
*Hi Jun, Soo sorry for the typo/mistake. On 02/12/2016, 11:19, "Michael Pearce" wrote: Hi Jao Thanks for the response. Sorry for slow reply, both with personal sickness and also battling some critical issues encountered since upgrading to 0.10.1.0 1) Thans for spotting,

[jira] [Commented] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715179#comment-15715179 ] Alexey Ozeritskiy commented on KAFKA-4205: -- We see this on 0.10.1. ~20-30 errors

[jira] [Updated] (KAFKA-2247) Merge kafka.utils.Time and kafka.common.utils.Time

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2247: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request 2095

[GitHub] kafka pull request #2095: KAFKA-2247: Merge kafka.utils.Time and kafka.commo...

2016-12-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2095 --- 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] [Commented] (KAFKA-2247) Merge kafka.utils.Time and kafka.common.utils.Time

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715202#comment-15715202 ] ASF GitHub Bot commented on KAFKA-2247: --- Github user asfgit closed the pull request

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-02 Thread Michael Andre Pearce (IG) (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715205#comment-15715205 ] Michael Andre Pearce (IG) commented on KAFKA-4477: -- What we see as per th

[jira] [Commented] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-12-02 Thread Mark Shelton (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715221#comment-15715221 ] Mark Shelton commented on KAFKA-4322: - >> "We tend not to have such callbacks" The St

[jira] [Created] (KAFKA-4479) Streams tests should pass without hardcoded Time.SYSTEM in GroupCoordinator

2016-12-02 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4479: -- Summary: Streams tests should pass without hardcoded Time.SYSTEM in GroupCoordinator Key: KAFKA-4479 URL: https://issues.apache.org/jira/browse/KAFKA-4479 Project: Kafka

[jira] [Comment Edited] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715235#comment-15715235 ] Ismael Juma edited comment on KAFKA-4205 at 12/2/16 2:16 PM: -

[jira] [Commented] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715235#comment-15715235 ] Ismael Juma commented on KAFKA-4205: cc [~becket_qin] [~hachikuji] > NullPointerExcep

[jira] [Updated] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4205: --- Fix Version/s: 0.10.1.1 > NullPointerException in fetchOffsetsBefore > ---

[jira] [Updated] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4205: --- Labels: reliability (was: ) > NullPointerException in fetchOffsetsBefore > --

[jira] [Commented] (KAFKA-4477) Node reduces its ISR to itself, and doesn't recover. Other nodes do not take leadership, cluster remains sick until node is restarted.

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715279#comment-15715279 ] Ismael Juma commented on KAFKA-4477: Stacktraces would be useful indeed. > Node reduc

[jira] [Created] (KAFKA-4480) kafka-configs will execute the removal of an invalid property and not report an error

2016-12-02 Thread Justin (JIRA)
Justin created KAFKA-4480: - Summary: kafka-configs will execute the removal of an invalid property and not report an error Key: KAFKA-4480 URL: https://issues.apache.org/jira/browse/KAFKA-4480 Project: Kafka

Re: [VOTE] KIP-84: Support SASL SCRAM mechanisms

2016-12-02 Thread Ismael Juma
Thanks Rajini. Let's see what Jun says about limiting the number of SHA variants. Either way, +1 from me. Ismael On Fri, Dec 2, 2016 at 2:40 PM, Rajini Sivaram wrote: > Ismael, > > 1. Jun had suggested added the full list of SHA-nnn in the [DISCUSS] > thread. I am ok with limiting to a smaller

[jira] [Commented] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-12-02 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715322#comment-15715322 ] Damian Guy commented on KAFKA-4322: --- My only concern with adding the extra methods is th

[jira] [Commented] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715331#comment-15715331 ] ASF GitHub Bot commented on KAFKA-4205: --- GitHub user ataraxer opened a pull request:

[GitHub] kafka pull request #2204: KAFKA-4205; KafkaApis: fix NPE caused by conversio...

2016-12-02 Thread ataraxer
GitHub user ataraxer opened a pull request: https://github.com/apache/kafka/pull/2204 KAFKA-4205; KafkaApis: fix NPE caused by conversion to array NPE was caused by `log.logSegments.toArray` resulting in array containing `null` values. The exact reason still remains somewhat a myste

[jira] [Updated] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexey Ozeritskiy updated KAFKA-4205: - Assignee: Anton Karamanov > NullPointerException in fetchOffsetsBefore > -

Re: [VOTE] KIP-84: Support SASL SCRAM mechanisms

2016-12-02 Thread Rajini Sivaram
Ismael, 1. Jun had suggested added the full list of SHA-nnn in the [DISCUSS] thread. I am ok with limiting to a smaller number if required. 3. Added a section on security considerations to the KIP. Thank you, Rajini On Thu, Dec 1, 2016 at 4:22 PM, Ismael Juma wrote: > Hi Rajini, > > Sorry fo

[jira] [Updated] (KAFKA-4205) NullPointerException in fetchOffsetsBefore

2016-12-02 Thread Alexey Ozeritskiy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexey Ozeritskiy updated KAFKA-4205: - Status: Patch Available (was: Open) > NullPointerException in fetchOffsetsBefore > --

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread radai
"wouldn't you be in the business of making sure everyone uses them properly?" thats where a broker-side plugin would come handy - any incoming message that does not conform to org policy (read - does not have the proper headers) gets thrown out (with an error returned to user) On Thu, Dec 1, 2016

[jira] [Commented] (KAFKA-4473) KafkaStreams does *not* guarantee at-least-once delivery

2016-12-02 Thread Thomas Schulz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715394#comment-15715394 ] Thomas Schulz commented on KAFKA-4473: -- To 1.) Even if the retries are in fact "0", t

[jira] [Commented] (KAFKA-4473) KafkaStreams does *not* guarantee at-least-once delivery

2016-12-02 Thread Thomas Schulz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715388#comment-15715388 ] Thomas Schulz commented on KAFKA-4473: -- 1. I did not configure the retries explicitly

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

2016-12-02 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-2247; Merge kafka.utils.Time and kafka.common.utils.Time -- [...truncated 14450 lines...] org.apache.kafka.streams.KafkaStreamsTest > shouldReturnFalseOnCloseWhenThr

[jira] [Commented] (KAFKA-4473) KafkaStreams does *not* guarantee at-least-once delivery

2016-12-02 Thread Thomas Schulz (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15715586#comment-15715586 ] Thomas Schulz commented on KAFKA-4473: -- UPDATE: I added the following config: stream

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-02 Thread Mayuresh Gharat
Hi Michael, 5) I believe this was in the discussion thread, @Mayuresh is this something we’ve overlooked? I thought we would down convert and remove the value so the old consumer had existing behavior, or is there something we haven’t thought about? ---> Yea, we would have to down convert and remo

[GitHub] kafka pull request #2198: HOTFIX: Forgot one transition

2016-12-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2198 --- 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

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-02 Thread Jun Rao
Hi, Michael, For 2), this is fine. Could you update the KIP wiki to make this and other points clearer? Other than that, the KIP looks good to me. An orthogonal thing is that there are other KIPs such as KIP-98 that also intend to change the message format. If they all get approved, we should thi

[jira] [Work started] (KAFKA-4456) Offsets of deleted topics are not removed from consumer groups

2016-12-02 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4456 started by Vahid Hashemian. -- > Offsets of deleted topics are not removed from consumer groups > --

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

2016-12-02 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-2247; Merge kafka.utils.Time and kafka.common.utils.Time -- [...truncated 14223 lines...] org.apache.kafka.streams.processor.TopologyBuilderTest > shouldNotAllowNul

[GitHub] kafka pull request #2202: MINOR: Improvements in group metadata cleanup and ...

2016-12-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2202 --- 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] [Commented] (KAFKA-4322) StateRestoreCallback begin and end indication

2016-12-02 Thread Mark Shelton (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716086#comment-15716086 ] Mark Shelton commented on KAFKA-4322: - I do not at all like the idea of using the metr

[jira] [Commented] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716114#comment-15716114 ] Mayuresh Gharat commented on KAFKA-4454: [~ijuma] I might not be understanding "ac

[jira] [Comment Edited] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716114#comment-15716114 ] Mayuresh Gharat edited comment on KAFKA-4454 at 12/2/16 8:00 PM: ---

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-02 Thread Guozhang Wang
@Jay 1. Stream's applicationId is shared among all instances for the app, and is used as part of the consumer group id, while "app.id" is per producer instance. So a Streams app that has a single "applicationID" config will likely contain multiple producers each with a different appID based on the

[jira] [Created] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
Xavier Léauté created KAFKA-4481: Summary: Relax Kafka Streams API type constraints Key: KAFKA-4481 URL: https://issues.apache.org/jira/browse/KAFKA-4481 Project: Kafka Issue Type: Improvemen

[jira] [Commented] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716140#comment-15716140 ] Xavier Léauté commented on KAFKA-4481: -- This change should be backwards-compatible an

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

2016-12-02 Thread Guozhang Wang
@Henry Seems like you cannot wait for the Streams EOS KIP any more :P We do not make assumptions on the underlying state store's transaction semantics, since it may not be necessarily RocksDB as users could customize their own store engines. As I mentioned, in the worst case we will invalidate t

[GitHub] kafka pull request #2205: KAFKA-4481: relax streams api type contraints

2016-12-02 Thread xvrl
GitHub user xvrl opened a pull request: https://github.com/apache/kafka/pull/2205 KAFKA-4481: relax streams api type contraints Make appropriate methods contravariant in key and value types. You can merge this pull request into a Git repository by running: $ git pull https://gi

[jira] [Commented] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716152#comment-15716152 ] ASF GitHub Bot commented on KAFKA-4481: --- GitHub user xvrl opened a pull request:

[jira] [Comment Edited] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716114#comment-15716114 ] Mayuresh Gharat edited comment on KAFKA-4454 at 12/2/16 8:13 PM: ---

Re: [DISCUSS] KIP-96 - Add per partition metrics for in-sync and assigned replica count

2016-12-02 Thread Gwen Shapira
All replicas are assigned replicas. Calling it AssignedReplicasCount makes it sound like we have non-assigned replicas that we are also counting somewhere. (Guess who requested the change...) On Fri, Dec 2, 2016 at 5:25 AM, Ismael Juma wrote: > Hi Xavier, > > Can you please share the reasoning f

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread Gwen Shapira
Daniel, Can you share specifics of how you'd use headers (the same way Todd did)? I think it may help the discussion. Thanks! On Fri, Dec 2, 2016 at 12:30 AM, Daniel Schierbeck wrote: > I don't have a lot of feedback on this, but at Zendesk we could definitely > use a standardized header system

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread Gwen Shapira
Woah, I wasn't aware this is something we'll do. It wasn't in the KIP, right? I guess we could do it the same way ACLs currently work. I had in mind something that will allow admins to apply rules to the new create/delete/config topic APIs. So Todd can decide to reject "create topic" requests that

Re: [DISCUSS] 0.10.1.1 Plan

2016-12-02 Thread Guozhang Wang
@Sean, There have been some discussions about KAFKA-4250, from Ismael. The main concern is on backward compatibility between 0.10.1.0 and the coming 0.10.1.1. Status Update: We are having three tasks left for 0.10.1.1, all of which have a PR under review and close to be merged. After those thre

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

2016-12-02 Thread Apache Jenkins Server
See Changes: [wangguoz] 2198 -- [...truncated 14383 lines...] org.apache.kafka.streams.StreamsConfigTest > shouldSupportPrefixedPropertiesThatAreNotPartOfRestoreConsumerConfig STARTED org.apa

Build failed in Jenkins: kafka-0.10.1-jdk7 #98

2016-12-02 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Improvements in group metadata cleanup and test coverage -- [...truncated 5550 lines...] kafka.api.AuthorizerIntegrationTest > testPatternSubscriptionWithTopicAnd

[jira] [Created] (KAFKA-4482) Kafka streams test failure and hang causing PR builds to fail

2016-12-02 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-4482: - Summary: Kafka streams test failure and hang causing PR builds to fail Key: KAFKA-4482 URL: https://issues.apache.org/jira/browse/KAFKA-4482 Project: Kafka

Re: [DISCUSS] KIP-97: Improved Kafka Client RPC Compatibility Policy

2016-12-02 Thread Ashish Singh
Colin, I just rebased KAFKA-3600's PR on trunk. KAFKA-4457 is a good idea, however it is probably doing some redundant work. AdminClient has a network client that will already have api versions info (after KAFKA-3600 goes in), so I do not think we need to send out another ApiVersions request over

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread radai
this KIP is about enabling headers, nothing more nothing less - so no, broker-side use of headers is not in the KIP scope. obviously though, once you have headers potential use cases could include broker-side header-aware interceptors (which would be the topic of other future KIPs). a trivially c

[jira] [Commented] (KAFKA-4482) Kafka streams test failure and hang causing PR builds to fail

2016-12-02 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716623#comment-15716623 ] Ismael Juma commented on KAFKA-4482: There is a PR to disable these tests, but [~guozh

[jira] [Commented] (KAFKA-4482) Kafka streams test failure and hang causing PR builds to fail

2016-12-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716675#comment-15716675 ] Matthias J. Sax commented on KAFKA-4482: Thanks for reporting. There is already a

[jira] [Resolved] (KAFKA-4482) Kafka streams test failure and hang causing PR builds to fail

2016-12-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-4482. Resolution: Duplicate > Kafka streams test failure and hang causing PR builds to fail >

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-12-02 Thread Jun Rao
Since this KIP affects message format, wire protocol, apis, I think it's worth spending a bit more time to nail down the concrete use cases. It would be bad if we add this feature, but when start implementing it for say mirroring, we then realize that header is not the best approach. Initially, I t

[jira] [Updated] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-4481: - Description: Streams API methods that apply transformations to streams are currently invariant in

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

2016-12-02 Thread Apache Jenkins Server
See Changes: [wangguoz] 2198 -- [...truncated 14238 lines...] org.apache.kafka.streams.processor.TopologyBuilderTest > shouldThroughOnUnassignedStateStoreAccess PASSED org.apache.kafka.streams

Re: [VOTE] KIP-93: Improve invalid timestamp handling in Kafka Streams

2016-12-02 Thread Matthias J. Sax
I am closing this vote now. KIP-93 is accepted with +3 (binding) +2 (non-binding) Thanks for feedback on DISCUSS thread and voting. -Matthias On 11/30/16 9:29 AM, Sriram Subramanian wrote: > +1 (binding) > > I second Ewen's point about compatibility. > > On Wed, Nov 30, 2016 at 2:53 AM, Dami

[jira] [Updated] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-4481: - Issue Type: Bug (was: Improvement) > Relax Kafka Streams API type constraints > -

[jira] [Work started] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4481 started by Xavier Léauté. > Relax Kafka Streams API type constraints >

[jira] [Work started] (KAFKA-4458) Add per partition metrics for in-sync and assigned replica count

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4458 started by Xavier Léauté. > Add per partition metrics for in-sync and assigned replica count >

[jira] [Updated] (KAFKA-4481) Relax Kafka Streams API type constraints

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-4481: - Status: Patch Available (was: In Progress) > Relax Kafka Streams API type constraints > -

[jira] [Updated] (KAFKA-4458) Add per partition metrics for in-sync and assigned replica count

2016-12-02 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-4458: - Status: Patch Available (was: In Progress) > Add per partition metrics for in-sync and assigned r

[jira] [Commented] (KAFKA-4473) KafkaStreams does *not* guarantee at-least-once delivery

2016-12-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716796#comment-15716796 ] Guozhang Wang commented on KAFKA-4473: -- Thanks for the update [~Thomas Schulz]. I thi

[jira] [Updated] (KAFKA-4473) RecordCollector should handle retriable exceptions more strictly

2016-12-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4473: - Summary: RecordCollector should handle retriable exceptions more strictly (was: KafkaStreams does

[jira] [Updated] (KAFKA-4473) RecordCollector should handle retriable exceptions more strictly

2016-12-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4473?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4473: - Labels: architecture (was: ) > RecordCollector should handle retriable exceptions more strictly >

[jira] [Created] (KAFKA-4483) Exception when creating topics

2016-12-02 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4483: -- Summary: Exception when creating topics Key: KAFKA-4483 URL: https://issues.apache.org/jira/browse/KAFKA-4483 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-4484) Set more conservative default values on RocksDB for memory usage

2016-12-02 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4484: Summary: Set more conservative default values on RocksDB for memory usage Key: KAFKA-4484 URL: https://issues.apache.org/jira/browse/KAFKA-4484 Project: Kafka

[jira] [Commented] (KAFKA-4484) Set more conservative default values on RocksDB for memory usage

2016-12-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4484?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716836#comment-15716836 ] Guozhang Wang commented on KAFKA-4484: -- [~h...@pinterest.com] [~flyaruu] do you want

[jira] [Updated] (KAFKA-4484) Set more conservative default values on RocksDB for memory usage

2016-12-02 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4484?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4484: - Assignee: (was: Henry Cai) > Set more conservative default values on RocksDB for memory usage

[GitHub] kafka pull request #2206: KAFKA-4454 : Authorizer should also include the Pr...

2016-12-02 Thread MayureshGharat
GitHub user MayureshGharat opened a pull request: https://github.com/apache/kafka/pull/2206 KAFKA-4454 : Authorizer should also include the Principal generated by the PrincipalBuilder. Added support to include the Principal generated by the PrincipalBuilder in KafkaChannel You can

[jira] [Commented] (KAFKA-4454) Authorizer should also include the Principal generated by the PrincipalBuilder.

2016-12-02 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15716846#comment-15716846 ] ASF GitHub Bot commented on KAFKA-4454: --- GitHub user MayureshGharat opened a pull re

  1   2   >