Re: Build problems in Eclipse dev environment setup

2017-01-06 Thread Dhwani Katagade
Hi, I was able to resolve the compile errors in Eclipse by manually setting the Default output folder for core project to core/build/classes/main. Looking further into this issue I see that the output folders for gradle and those for eclipse get configured differently. The output folders in

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Dong Lin
Hey Mayuresh, Thanks for the comment. If the message's offset is below low_watermark, then it should have been deleted by log retention policy. Thus it is OK not to expose this message to consumer. Does this answer your question? Thanks, Dong On Fri, Jan 6, 2017 at 4:21 PM, Mayuresh Gharat

[jira] [Assigned] (KAFKA-3502) Build is killed during kafka streams tests due to `pure virtual method called` error

2017-01-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-3502: Assignee: Guozhang Wang > Build is killed during kafka streams tests due to `pure virtual

Re: KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Roger Hoover
Thank you, Ismael. Sent from my iPhone > On Jan 6, 2017, at 4:46 PM, Ismael Juma wrote: > > Thanks Roger. I asked around and it seems like `listener name` is what > people found most intuitive in the context of configs. So, I have updated > the KIP to use that. > > Ismael >

[jira] [Updated] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4607: - Assignee: Nikki Thean > Kafka Streams allows you to provide strings with illegal characters for

[jira] [Comment Edited] (KAFKA-4558) throttling_test fails if the producer starts too fast.

2017-01-06 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806421#comment-15806421 ] Apurva Mehta edited comment on KAFKA-4558 at 1/7/17 2:00 AM: - So I had a look

[jira] [Commented] (KAFKA-4558) throttling_test fails if the producer starts too fast.

2017-01-06 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806466#comment-15806466 ] Apurva Mehta commented on KAFKA-4558: - I dug a bit further into the `JmxMixin`, and it seems that all

[jira] [Commented] (KAFKA-3502) Build is killed during kafka streams tests due to `pure virtual method called` error

2017-01-06 Thread Stevo Slavic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806437#comment-15806437 ] Stevo Slavic commented on KAFKA-3502: - This is happening every time when [PR

[jira] [Commented] (KAFKA-4558) throttling_test fails if the producer starts too fast.

2017-01-06 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806421#comment-15806421 ] Apurva Mehta commented on KAFKA-4558: - So I had a look at the code. All the 13 tests which use

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Ismael Juma
Hi all, Since a few people (including myself) felt that listener name was clearer than protocol label, I updated the KIP to use that (as mentioned in the discuss thread). Given that this is a minor change, I don't think we need to restart the vote. If anyone objects to this change, please let me

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Sriram Subramanian
+1 On Fri, Jan 6, 2017 at 4:41 PM, Eno Thereska wrote: > I light of the recent discussion on KIP-104 I've made changes to the KIP > to explicitly list all APIs of the Metrics class for completion. I think > it's fair to re-start the vote on the KIP in light of the new

Re: KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Ismael Juma
Thanks Roger. I asked around and it seems like `listener name` is what people found most intuitive in the context of configs. So, I have updated the KIP to use that. Ismael On Fri, Jan 6, 2017 at 9:42 PM, Roger Hoover wrote: > Ismael, > > Listener id would also convey

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
I light of the recent discussion on KIP-104 I've made changes to the KIP to explicitly list all APIs of the Metrics class for completion. I think it's fair to re-start the vote on the KIP in light of the new discussion for transparency:

Problem generating documentation locally

2017-01-06 Thread Vahid S Hashemian
I'm trying to build the project documentation locally, but I can't seem to be able to populate content for files in the 'docs/generated/' folder. I've tried some of the doc-related build commands from https://github.com/apache/kafka/blob/trunk/README.md but they just seem to create the files in

[jira] [Commented] (KAFKA-4606) make getOffsetsTopicPartitionCount mutable

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4606?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806263#comment-15806263 ] Ewen Cheslack-Postava commented on KAFKA-4606: -- Oh, yeah, I definitely think it'd nice to be

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Mayuresh Gharat
Hi Dong, Thanks for the KIP. I had a question (which might have been answered before). 1) The KIP says that the low_water_mark will be updated periodically by the broker like high_water_mark. Essentially we want to use low_water_mark for cases where an entire segment cannot be deleted because

Re: [VOTE] KIP-88: OffsetFetch Protocol Update

2017-01-06 Thread Vahid S Hashemian
I'd like to thank everyone who took the time to review, provide feedback, and vote on this KIP. The KIP passed with 4 binding and 3 non-binding +1 votes. Binding Votes * Gwen Shapira * Jason Gustafson * Ismael Juma * Ewen Cheslack-Postava Non-Binding Votes: * Edoardo Comar * Mickael Maison *

[jira] [Commented] (KAFKA-4606) make getOffsetsTopicPartitionCount mutable

2017-01-06 Thread Ryan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4606?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806253#comment-15806253 ] Ryan P commented on KAFKA-4606: --- Artificial was the wrong word, but I still think an argument can be made to

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
Ok, I'll list all the methods in the Metrics class for completion. An interface won't work since it will have to reside outside of streams unfortunately. Thanks Eno > On 6 Jan 2017, at 23:54, Ismael Juma wrote: > > Hi Guozhang, > > I understand the requirement and I don't

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Ismael Juma
Hi Guozhang, I understand the requirement and I don't have an issue with that. My point is that the `Metrics` registry API is becoming public via this KIP so we should ensure that it's suitable. It may make sense to introduce an interface (say MetricsRegistry) that exposes a reasonable subset (do

[jira] [Commented] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806176#comment-15806176 ] Matthias J. Sax commented on KAFKA-4607: Very good catch! > Kafka Streams allows you to provide

[jira] [Commented] (KAFKA-2273) Add rebalance with a minimal number of reassignments to server-defined strategy list

2017-01-06 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806172#comment-15806172 ] Vahid Hashemian commented on KAFKA-2273: No new update since my last update on Dec 21. It still

[jira] [Updated] (KAFKA-4465) Create docker image and scripts for running tests locally

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4465?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4465: - Status: Patch Available (was: Open) > Create docker image and scripts for

[jira] [Updated] (KAFKA-4463) Setup travis-ci integration for ducktape tests

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4463: - Status: Patch Available (was: Open) > Setup travis-ci integration for ducktape

[jira] [Updated] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikki Thean updated KAFKA-4607: --- Description: When using the aggregate function of the Kafka Streams DSL, I supplied the function

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
Thanks Guozhang, I adjusted the KIP to explicitly mention that we are exposing the Metrics object now as Ismael suggested, since it was lost in the details. Eno > On 6 Jan 2017, at 22:16, Ismael Juma wrote: > > Thanks for the explanation Eno. The KIP did mention that the

[jira] [Commented] (KAFKA-2273) Add rebalance with a minimal number of reassignments to server-defined strategy list

2017-01-06 Thread Allen Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806146#comment-15806146 ] Allen Wang commented on KAFKA-2273: --- What is the plan for this JIRA/KIP? It is set to be fixed in 0.10.2

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Guozhang Wang
Unlike Producer and Consumer, Streams users may likely to add their own sensors depending on their apps and that is the main reason we added facilities to let them register customized "throughput" "latency" and any generic sensors. I think Eno has thought about just adding an API in

[jira] [Issue Comment Deleted] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikki Thean updated KAFKA-4607: --- Comment: was deleted (was: Potential fix: [https://github.com/apache/kafka/pull/2331]) > Kafka

[jira] [Commented] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806138#comment-15806138 ] ASF GitHub Bot commented on KAFKA-4607: --- GitHub user nixsticks opened a pull request:

[jira] [Commented] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Nikki Thean (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4607?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806141#comment-15806141 ] Nikki Thean commented on KAFKA-4607: Potential fix: [https://github.com/apache/kafka/pull/2331] >

[GitHub] kafka pull request #2331: KAFKA-4607: Validate the names of auto-generated i...

2017-01-06 Thread nixsticks
GitHub user nixsticks opened a pull request: https://github.com/apache/kafka/pull/2331 KAFKA-4607: Validate the names of auto-generated internal topics I considered catching errors to add further information about naming internal state stores. However, Topic.validate() will throw

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

2017-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: cleanup Kafka Streams exception classes -- [...truncated 8001 lines...] kafka.integration.SaslPlaintextTopicMetadataTest >

[jira] [Created] (KAFKA-4607) Kafka Streams allows you to provide strings with illegal characters for internal topic names

2017-01-06 Thread Nikki Thean (JIRA)
Nikki Thean created KAFKA-4607: -- Summary: Kafka Streams allows you to provide strings with illegal characters for internal topic names Key: KAFKA-4607 URL: https://issues.apache.org/jira/browse/KAFKA-4607

[jira] [Updated] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4576: - Status: Patch Available (was: Open) > Log segments close to max size break on

[jira] [Updated] (KAFKA-4039) Exit Strategy: using exceptions instead of inline invocation of exit/halt

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4039: - Status: Patch Available (was: Open) > Exit Strategy: using exceptions instead of

[jira] [Updated] (KAFKA-4602) KIP-72 Allow putting a bound on memory consumed by Incoming requests

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4602?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4602: - Fix Version/s: 0.10.2.0 Status: Patch Available (was: Open) > KIP-72

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2017-01-06 Thread radai
JIRA up - https://issues.apache.org/jira/browse/KAFKA-4602 PR up - https://github.com/apache/kafka/pull/2330 KIP wiki has been updated. On Fri, Jan 6, 2017 at 8:16 AM, radai wrote: > Will do (sorry for the delay). > and thank you. > > On Fri, Jan 6, 2017 at 7:56

[jira] [Commented] (KAFKA-4602) KIP-72 Allow putting a bound on memory consumed by Incoming requests

2017-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4602?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806020#comment-15806020 ] ASF GitHub Bot commented on KAFKA-4602: --- GitHub user radai-rosenblatt opened a pull request:

[GitHub] kafka pull request #2330: KAFKA-4602 - KIP-72 - Allow putting a bound on mem...

2017-01-06 Thread radai-rosenblatt
GitHub user radai-rosenblatt opened a pull request: https://github.com/apache/kafka/pull/2330 KAFKA-4602 - KIP-72 - Allow putting a bound on memory consumed by Incoming requests this is the initial implementation. You can merge this pull request into a Git repository by running:

[jira] [Commented] (KAFKA-4381) Add per partition lag metric to KafkaConsumer.

2017-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15806008#comment-15806008 ] ASF GitHub Bot commented on KAFKA-4381: --- GitHub user becketqin opened a pull request:

[GitHub] kafka pull request #2329: KAFKA-4381: Add per partition lag metrics to the c...

2017-01-06 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2329 KAFKA-4381: Add per partition lag metrics to the consumer You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka KAFKA-4381

[jira] [Commented] (KAFKA-4606) make getOffsetsTopicPartitionCount mutable

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4606?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805984#comment-15805984 ] Ewen Cheslack-Postava commented on KAFKA-4606: -- It's not really an artificial limitation

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Ismael Juma
Thanks for the explanation Eno. The KIP did mention that the metrics registry would be exposed, yes. What is missing is that the registry is not currently exposed by anything else. Traditionally, we list all public APIs created by a KIP, which is effectively true for the registry in this case. Did

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

2017-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3452: Support session windows -- [...truncated 31144 lines...] kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache PASSED

[jira] [Commented] (KAFKA-3264) Mark the old Scala consumer and related classes as deprecated

2017-01-06 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805864#comment-15805864 ] ASF GitHub Bot commented on KAFKA-3264: --- GitHub user vahidhashemian opened a pull request:

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

2017-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: cleanup Kafka Streams exception classes -- [...truncated 7994 lines...] kafka.log.LogTest > testReadWithTooSmallMaxLength STARTED kafka.log.LogTest >

[GitHub] kafka pull request #2328: KAFKA-3264: Deprecate the old Scala consumer (WIP)

2017-01-06 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2328 KAFKA-3264: Deprecate the old Scala consumer (WIP) You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka KAFKA-3264

Re: KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Roger Hoover
Ismael, Listener id would also convey uniqueness but I'm ok with listener key as well since it fits with the use of the term "map" in other properties. My initially feeling against the word key was that it seemed more natural in documentation about Kafka allowing multiple listener (even with

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Becket Qin
+1 Thanks for the proposal. On Fri, Jan 6, 2017 at 11:37 AM, Roger Hoover wrote: > +1 (non-binding) > > On Fri, Jan 6, 2017 at 11:16 AM, Tom Crayford > wrote: > > > +1 (non-binding) > > > > On Fri, Jan 6, 2017 at 6:58 PM, Colin McCabe

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Eno Thereska
So the KIP proposes exposing the metrics registry (second paragraph under motivation). The community has indicated that they would like to 1. access all the metrics and 2. register their own. We provide some helper interfaces for them to register throughput and latency metrics, but ultimately

[GitHub] kafka pull request #2327: MINOR: Fix small error in javadoc for persistent S...

2017-01-06 Thread nixsticks
GitHub user nixsticks opened a pull request: https://github.com/apache/kafka/pull/2327 MINOR: Fix small error in javadoc for persistent Stores Probably copy/pasted from the factory for in-memory stores above it :) You can merge this pull request into a Git repository by running:

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Ismael Juma
Thanks for the KIP. Sounds useful. One thing that wasn't made clear is that we are exposing `Metrics` as a public class for the first time. Neither the consumer or producer expose it at the moment. Do we want to expose the whole class or would it be better to expose a more limited interface?

Re: KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Ismael Juma
Hi Roger, I think `listener_key` makes it clear that it has to be unique per listener, so I prefer it a little over `listener_name`. Since the existing config is called `listeners` instead of `protocol.listeners`, maybe we don't need the protocol prefix? Ismael On Fri, Jan 6, 2017 at 7:48 PM,

[jira] [Resolved] (KAFKA-1908) Split brain

2017-01-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1908?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-1908. -- Resolution: Not A Problem > Split brain > --- > > Key: KAFKA-1908 >

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

2017-01-06 Thread Apache Jenkins Server
See

Re: Different key with the same digest in log compaction

2017-01-06 Thread radai
i just noticed my link didnt parse correctly. the formula for probability of collision is explained by googling "birthday paradox". if anything, we could further optimize this code to use a trivial hash map for cases where sizeOf(hash) > sizeOf(key) On Fri, Jan 6, 2017 at 10:00 AM, Colin McCabe

Re: [VOTE] KIP-102 - Add close with timeout for consumers

2017-01-06 Thread Apurva Mehta
+1 (non-binding). On Fri, Jan 6, 2017 at 9:24 AM, Jason Gustafson wrote: > Thanks for the KIP. +1 > > On Fri, Jan 6, 2017 at 2:26 AM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > As I said in the discussion thread, I'm not too sure

Re: KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Roger Hoover
Maybe it's clearer to to say protocol_listener_name? The proposed config allows you to name each listener and refer to their names in various places. On Wed, Jan 4, 2017 at 4:34 AM, Ismael Juma wrote: > Hi Colin, > > Thanks for the feedback. It's a good question regarding

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

2017-01-06 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-4606) make getOffsetsTopicPartitionCount mutable

2017-01-06 Thread Ryan P (JIRA)
Ryan P created KAFKA-4606: - Summary: make getOffsetsTopicPartitionCount mutable Key: KAFKA-4606 URL: https://issues.apache.org/jira/browse/KAFKA-4606 Project: Kafka Issue Type: Improvement

Re: [VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-06 Thread Shikhar Bhushan
That makes sense to me, I'll fold that into the PR and update the KIP if it gets committed in that form. On Fri, Jan 6, 2017 at 9:44 AM Jason Gustafson wrote: > +1 One minor comment: would it make sense to let the `Transformation` > interface extend `o.a.k.c.Configurable`

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Roger Hoover
+1 (non-binding) On Fri, Jan 6, 2017 at 11:16 AM, Tom Crayford wrote: > +1 (non-binding) > > On Fri, Jan 6, 2017 at 6:58 PM, Colin McCabe wrote: > > > Looks good. +1 (non-binding). > > > > What do you think about changing "protocol label" to "listener

[jira] [Commented] (KAFKA-4598) Create new SourceTask commit callback method that takes offsets param

2017-01-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805460#comment-15805460 ] Shikhar Bhushan commented on KAFKA-4598: Yeah, that's a reasonable alternative with the caveat you

[jira] [Commented] (KAFKA-4598) Create new SourceTask commit callback method that takes offsets param

2017-01-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805375#comment-15805375 ] Ewen Cheslack-Postava commented on KAFKA-4598: -- The alternative would be to synchronize them,

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Dong Lin
Hey Jun, Thanks for reviewing the KIP! 1. The low_watermark will be checkpointed in a new file named "replication-low-watermark-checkpoint". It will have the same format as the existing replication-offset-checkpoint file. This allows us the keep the existing format of checkpoint files which

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Tom Crayford
+1 (non-binding) On Fri, Jan 6, 2017 at 6:58 PM, Colin McCabe wrote: > Looks good. +1 (non-binding). > > What do you think about changing "protocol label" to "listener key"? > > best, > Colin > > > On Fri, Jan 6, 2017, at 09:23, Neha Narkhede wrote: > > +1 > > > > On Fri,

[jira] [Work started] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-01-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4222 started by Matthias J. Sax. -- > Transient failure in QueryableStateIntegrationTest.queryOnRebalance >

[jira] [Assigned] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-01-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4222: -- Assignee: Matthias J. Sax (was: Eno Thereska) > Transient failure in

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Colin McCabe
Looks good. +1 (non-binding). What do you think about changing "protocol label" to "listener key"? best, Colin On Fri, Jan 6, 2017, at 09:23, Neha Narkhede wrote: > +1 > > On Fri, Jan 6, 2017 at 9:21 AM Jun Rao wrote: > > > Hi, Ismael, > > > > Thanks for the KIP. +1 > >

[GitHub] kafka pull request #2326: MINOR: Various small scala cleanups

2017-01-06 Thread mimaison
GitHub user mimaison opened a pull request: https://github.com/apache/kafka/pull/2326 MINOR: Various small scala cleanups - Removed unnecessary semicolons You can merge this pull request into a Git repository by running: $ git pull https://github.com/mimaison/kafka minor-fixes

[jira] [Commented] (KAFKA-3452) Support session windows

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

[GitHub] kafka pull request #2166: KAFKA-3452: Support session windows

2017-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2166 --- 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] [Updated] (KAFKA-3452) Support session windows

2017-01-06 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3452: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-4381) Add per partition lag metric to KafkaConsumer.

2017-01-06 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805168#comment-15805168 ] Jiangjie Qin commented on KAFKA-4381: - [~ijuma] Thanks for the reminder :) I'll submit patch this

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2017-01-06 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805165#comment-15805165 ] Jiangjie Qin commented on KAFKA-4497: - [~michael.andre.pearce] Thanks for the update. Glad to know it

Re: [DISCUSS] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-06 Thread Jun Rao
Hi, Dong, Thanks for the proposal. Looks good overall. A couple of comments. 1. Where is the low_watermark checkpointed? Is that in replication-offset-checkpoint? If so, do we need to bump up the version? Could you also describe the format change? 2. For topics with "delete" retention,

Re: Different key with the same digest in log compaction

2017-01-06 Thread Colin McCabe
That's a fair point. The calls to Arrays.equals are comparing just the hashes, not the keys. Colin On Tue, Jan 3, 2017, at 17:15, radai wrote: > looking at the code (SkimpyOffsetMap.get/put) they both start with > hashInto(key, hash1) and then ignore key from that point on - so we're > not >

Re: [VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-06 Thread Jason Gustafson
+1 One minor comment: would it make sense to let the `Transformation` interface extend `o.a.k.c.Configurable` and remove the `init` method? On Thu, Jan 5, 2017 at 5:48 PM, Neha Narkhede wrote: > +1 (binding) > > On Wed, Jan 4, 2017 at 2:36 PM Shikhar Bhushan

Re: KafkaConnect SinkTask::put

2017-01-06 Thread Shikhar Bhushan
Sorry I forgot to specify, this needs to go into your Connect worker configuration. On Fri, Jan 6, 2017 at 02:57 wrote: > Hi Shikhar, > > I've just added this to ~config/consumer.properties in the Kafka folder > but it doesn't appear to have made any difference. Have I

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Sriram Subramanian
+1 On Fri, Jan 6, 2017 at 9:12 AM, Matthias J. Sax wrote: > +1 > > On 1/6/17 8:01 AM, Guozhang Wang wrote: > > +1 > > > > On Fri, Jan 6, 2017 at 5:05 AM, Bill Bejeck wrote: > > > >> +1 > >> > >> On Fri, Jan 6, 2017 at 5:57 AM, Damian Guy

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-01-06 Thread Ismael Juma
Thanks for the review Jun. Yes, that's a good point, I have updated the KIP. Ismael On Fri, Jan 6, 2017 at 5:15 PM, Jun Rao wrote: > Hi, Ismael, > > Thanks for the KIP. Looks reasonable to me. To be consistent with the > pattern used in other pluggable interfaces, we

Re: [VOTE] KIP-102 - Add close with timeout for consumers

2017-01-06 Thread Jason Gustafson
Thanks for the KIP. +1 On Fri, Jan 6, 2017 at 2:26 AM, Ismael Juma wrote: > Thanks for the KIP, +1 (binding). > > As I said in the discussion thread, I'm not too sure about the hardcoded 30 > seconds timeout for the no-args `close` method. Still, it's an improvement > over

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Neha Narkhede
+1 On Fri, Jan 6, 2017 at 9:21 AM Jun Rao wrote: > Hi, Ismael, > > Thanks for the KIP. +1 > > Jun > > On Fri, Jan 6, 2017 at 2:51 AM, Ismael Juma wrote: > > > Hi all, > > > > As the discussion seems to have settled down, I would like to initiate > the > >

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

2017-01-06 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: update JavaDoc for simple helper interfaces of KStream and KTable -- [...truncated 3914 lines...] kafka.integration.SaslPlaintextTopicMetadataTest >

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Jun Rao
Hi, Ismael, Thanks for the KIP. +1 Jun On Fri, Jan 6, 2017 at 2:51 AM, Ismael Juma wrote: > Hi all, > > As the discussion seems to have settled down, I would like to initiate the > voting process for KIP-103: Separation of Internal and External traffic: > >

[jira] [Commented] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-01-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805036#comment-15805036 ] Matthias J. Sax commented on KAFKA-4222: One more instance:

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-01-06 Thread Jun Rao
Hi, Ismael, Thanks for the KIP. Looks reasonable to me. To be consistent with the pattern used in other pluggable interfaces, we probably should make the new interface configurable and closable? Jun On Fri, Jan 6, 2017 at 4:16 AM, Ismael Juma wrote: > Thanks Dan and Colin

Re: [VOTE] KIP-104: Granular Sensors for Streams

2017-01-06 Thread Matthias J. Sax
+1 On 1/6/17 8:01 AM, Guozhang Wang wrote: > +1 > > On Fri, Jan 6, 2017 at 5:05 AM, Bill Bejeck wrote: > >> +1 >> >> On Fri, Jan 6, 2017 at 5:57 AM, Damian Guy wrote: >> >>> +1 >>> >>> On Fri, 6 Jan 2017 at 09:37 Eno Thereska

Re: [VOTE] KIP-105: Addition of Record Level for Sensors

2017-01-06 Thread Matthias J. Sax
+1 On 1/6/17 9:09 AM, Neha Narkhede wrote: > +1 > > On Fri, Jan 6, 2017 at 9:04 AM Sriram Subramanian wrote: > >> +1 >> >> On Fri, Jan 6, 2017 at 8:40 AM, Bill Bejeck wrote: >> >>> +1 >>> >>> On Fri, Jan 6, 2017 at 11:06 AM, Guozhang Wang

Re: [VOTE] KIP-105: Addition of Record Level for Sensors

2017-01-06 Thread Neha Narkhede
+1 On Fri, Jan 6, 2017 at 9:04 AM Sriram Subramanian wrote: > +1 > > On Fri, Jan 6, 2017 at 8:40 AM, Bill Bejeck wrote: > > > +1 > > > > On Fri, Jan 6, 2017 at 11:06 AM, Guozhang Wang > wrote: > > > > > +1 > > > > > > On Fri, Jan 6,

Re: [VOTE] KIP-103: Separation of Internal and External traffic

2017-01-06 Thread Sriram Subramanian
+1 On Fri, Jan 6, 2017 at 3:21 AM, Rajini Sivaram wrote: > Ismael, > > Thank you for the KIP. It is a very useful feature. > > +1 (non-binding) > > Regards, > > Rajini > > On Fri, Jan 6, 2017 at 10:51 AM, Ismael Juma wrote: > > > Hi all, > > > > As

Re: [VOTE] KIP-105: Addition of Record Level for Sensors

2017-01-06 Thread Sriram Subramanian
+1 On Fri, Jan 6, 2017 at 8:40 AM, Bill Bejeck wrote: > +1 > > On Fri, Jan 6, 2017 at 11:06 AM, Guozhang Wang wrote: > > > +1 > > > > On Fri, Jan 6, 2017 at 2:55 AM, Damian Guy wrote: > > > > > +1 > > > > > > On Fri, 6 Jan 2017 at

[jira] [Commented] (KAFKA-4381) Add per partition lag metric to KafkaConsumer.

2017-01-06 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805002#comment-15805002 ] Ismael Juma commented on KAFKA-4381: [~becket_qin], reminder that the feature freeze is next Friday.

Re: [VOTE] KIP-105: Addition of Record Level for Sensors

2017-01-06 Thread Bill Bejeck
+1 On Fri, Jan 6, 2017 at 11:06 AM, Guozhang Wang wrote: > +1 > > On Fri, Jan 6, 2017 at 2:55 AM, Damian Guy wrote: > > > +1 > > > > On Fri, 6 Jan 2017 at 10:48 Ismael Juma wrote: > > > > > Thanks for the KIP, +1 (binding). > > > >

[GitHub] kafka pull request #2321: MINOR: update JavaDoc for simple helper interfaces...

2017-01-06 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2321 --- 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] [Comment Edited] (KAFKA-4441) Kafka Monitoring is incorrect during rapid topic creation and deletion

2017-01-06 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15804874#comment-15804874 ] Edoardo Comar edited comment on KAFKA-4441 at 1/6/17 4:16 PM: -- For the

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2017-01-06 Thread radai
Will do (sorry for the delay). and thank you. On Fri, Jan 6, 2017 at 7:56 AM, Ismael Juma wrote: > Radai, you have more than enough votes to declare the vote successful. > Maybe it's time to do so. :) Also, once you have done that, it would be > good to move this KIP to the

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

2017-01-06 Thread Rajini Sivaram
Oops, I had sent the following note, but it was sent as a response to Jun Rao and I didn't realize it wasn't sent to dev list. Thanks for pointing out, Ismael. The KIP has been moved to adopted list. *This vote has passed with three binding (Gwen, Jun, Ismael) and three non-binding votes

Re: [VOTE] KIP-105: Addition of Record Level for Sensors

2017-01-06 Thread Guozhang Wang
+1 On Fri, Jan 6, 2017 at 2:55 AM, Damian Guy wrote: > +1 > > On Fri, 6 Jan 2017 at 10:48 Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > Ismael > > > > On Fri, Jan 6, 2017 at 10:37 AM, Eno Thereska > > wrote:

  1   2   >