Re: [VOTE] 0.10.2.0 RC1

2017-02-14 Thread Moczarski, Swen
Thanks a lot for the fast fix! I tested my code with the fix and it works fine without exception. Regards, Swen Am 2/13/17, 8:38 PM schrieb "Guozhang Wang" : Thanks for reporting the JIRA Swen. Jason has a patch ready under KAFKA-4761 and I have reviewed it. You could try it ou

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

2017-02-14 Thread Guozhang Wang
Some thoughts on the mixture usage of DSL / PAPI: There were some suggestions on mixing the usage of DSL and PAPI: https://issues.apache.org/jira/browse/KAFKA-3455, and after thinking it a bit more carefully, I'd rather not recommend users following this pattern, since in DSL this can always be ac

Re: Correct prefetching of data to KTable-like structure on application startup

2017-02-14 Thread Jan Lukavský
Hi Matthias, I understand that the local cache will not be automatically cleared, but that is not an issue for me now. The problem I see is still the same as at the beginning - even caching data to RocksDB in KafkaStreams implementation might (I would say) suffer from this issue. When using

[jira] [Commented] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-14 Thread Prasanna Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15865486#comment-15865486 ] Prasanna Gautam commented on KAFKA-2729: This is still replicable in Kafka 0.10.1.

[jira] [Comment Edited] (KAFKA-2729) Cached zkVersion not equal to that in zookeeper, broker not recovering.

2017-02-14 Thread Prasanna Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15865486#comment-15865486 ] Prasanna Gautam edited comment on KAFKA-2729 at 2/14/17 9:49 AM: ---

Re: [VOTE] 0.10.2.0 RC1

2017-02-14 Thread Ian Duffy
Hi Matthias and Guozhang, +1 (non-binding) We eventually solved this on our end. It boiled down to dodgy data and unexpected hidden errors in our processing steps. This eventually resulted in a session time out which triggered the rebalances. Thanks again for all your help. Ian. On 14 February

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-14 Thread Eno Thereska
Even if users commit on every record, the expensive part will not be the checkpointing proposed in this KIP, but the rest of the commit. Eno > On 13 Feb 2017, at 23:46, Guozhang Wang wrote: > > I think I'm OK to always enable checkpointing, but I'm not sure if we want > to checkpoint on every

[jira] [Created] (KAFKA-4764) Improve diagnostics for SASL authentication failures

2017-02-14 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4764: - Summary: Improve diagnostics for SASL authentication failures Key: KAFKA-4764 URL: https://issues.apache.org/jira/browse/KAFKA-4764 Project: Kafka Issue Ty

[jira] [Commented] (KAFKA-4764) Improve diagnostics for SASL authentication failures

2017-02-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4764?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15865903#comment-15865903 ] ASF GitHub Bot commented on KAFKA-4764: --- GitHub user rajinisivaram opened a pull req

[GitHub] kafka pull request #2546: KAFKA-4764: Improve diagnostics for SASL auth fail...

2017-02-14 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/2546 KAFKA-4764: Improve diagnostics for SASL auth failures First step towards improving handling of client SASL authentication failures. You can merge this pull request into a Git repository by r

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-14 Thread Dong Lin
Hey Jun, I just realized that you may be suggesting that a tool for listing offline directories is necessary for KIP-112 by asking whether KIP-112 and KIP-113 will be in the same release. I think such a tool is useful but doesn't have to be included in KIP-112. This is because as of now admin need

[GitHub] kafka pull request #2547: MINOR: add session windows doc to streams.html

2017-02-14 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2547 MINOR: add session windows doc to streams.html You can merge this pull request into a Git repository by running: $ git pull https://github.com/dguy/kafka session-window-doc Alternatively you can

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

2017-02-14 Thread Mathieu Fenniak
On Tue, Feb 14, 2017 at 1:14 AM, Guozhang Wang wrote: > Some thoughts on the mixture usage of DSL / PAPI: > > There were some suggestions on mixing the usage of DSL and PAPI: > https://issues.apache.org/jira/browse/KAFKA-3455, and after thinking it a > bit more carefully, I'd rather not recommend

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

2017-02-14 Thread Damian Guy
> And about printing the topology for debuggability: I agrees this is a > > potential drawback, and I'd suggest maintain some functionality to build > a > > "dry topology" as Mathieu suggested; the difficulty is that, internally > we > > need a different "copy" of the topology for each thread so th

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

2017-02-14 Thread Mathieu Fenniak
On Tue, Feb 14, 2017 at 9:37 AM, Damian Guy wrote: > > And about printing the topology for debuggability: I agrees this is a > > > potential drawback, and I'd suggest maintain some functionality to > build > > a > > > "dry topology" as Mathieu suggested; the difficulty is that, internally > > we

get function

2017-02-14 Thread Samy CHBINOU
In kafka there is a Subscribe API to be notified of incoming message values from a topic. I didn't see a Get(key, topic) function. I mean a Get function to retreive a value from a key inside a topic. Isn't this feature implemented? (Like in Redis). If it doesn't exists, do you think is is easy

Request Wiki edit permission for mfenniak

2017-02-14 Thread Mathieu Fenniak
Hello, I'd like to contribute a KIP, and so I am requesting access to edit the Kafka Wiki for the user "mfenniak". Thanks, Mathieu

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

2017-02-14 Thread Stephen Durfey (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15866127#comment-15866127 ] Stephen Durfey commented on KAFKA-4159: --- I had a use case for this feature. I wanted

Re: Request Wiki edit permission for mfenniak

2017-02-14 Thread Guozhang Wang
Done, you can now also assign JIRAs to yourself. Cheers. Guozhang On Tue, Feb 14, 2017 at 8:51 AM, Mathieu Fenniak < mathieu.fenn...@replicon.com> wrote: > Hello, > > I'd like to contribute a KIP, and so I am requesting access to edit the > Kafka Wiki for the user "mfenniak". > > Thanks, > > Mat

kafka consumer coordinator is unknown issue

2017-02-14 Thread Ashwini Mhatre (asmhatre)
Hi, I have 3 node kafka cluster. I have one kafka consumer which will listen to particular topic. Facing following issue: 08:52:25.733 [SparkApp-akka.actor.default-dispatcher-2] DEBUG o.a.kafka.common.metrics.Metrics - Added sensor with name heartbeat-latency 08:52:25.733 [SparkApp-akka.actor.d

Request for JIRA access

2017-02-14 Thread Thomas Dutta
Dear Team, I am a Computer Science graduate student at the University of Illinois at Chicago. I am a newbie and I want to contribute to Apache Kafka project. Could you please add me to the list so that I can assign JIRA ticket to myself. Please let me know if you need additional information. Reg

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

2017-02-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15866212#comment-15866212 ] Jason Gustafson commented on KAFKA-4159: [~sjdurfey] I've added you as a contribut

[VOTE] KIP-82 Add Record Headers

2017-02-14 Thread Michael Pearce
Hi all, We would like to start the voting process for KIP-82 – Add record headers. The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers Discussion thread(s) can be found here: http://search-hadoop.com/m/Kafka/uyzND1nSTOHTvj81?subj=Re+DISCUSS+KIP

[jira] [Updated] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-14 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4762?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4762: --- Affects Version/s: 0.9.0.1 > Consumer throwing RecordTooLargeException even when messages are not that

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

2017-02-14 Thread Matthias J. Sax
You can already output any number of record within .transform() using the provided Context object from init()... -Matthias On 2/14/17 9:16 AM, Guozhang Wang wrote: >> and you can't output multiple records or branching logic from a > transform(); > > For output multiple records in transform, we

Re: [VOTE] KIP-82 Add Record Headers

2017-02-14 Thread radai
+1 from me. also - a more usable link to the discussion thread: http://markmail.org/message/x5wlkieexinovsz3 On Tue, Feb 14, 2017 at 9:42 AM, Michael Pearce wrote: > Hi all, > > We would like to start the voting process for KIP-82 – Add record headers. > The KIP can be found > at > > https://cw

[VOTE] 0.10.2.0 RC2

2017-02-14 Thread Ewen Cheslack-Postava
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 0.10.2.0. This is a minor version release of Apache Kafka. It includes 19 new KIPs. See the release notes and release plan (https://cwiki.apache.org/conf luence/display/KAFKA/Release+Plan+

[GitHub] kafka pull request #2535: MINOR: don't throw CommitFailedException during su...

2017-02-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2535 --- 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-4277) creating ephemeral node already exist

2017-02-14 Thread Wrikken (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15866415#comment-15866415 ] Wrikken commented on KAFKA-4277: Reproduced on our side with Scala 2.11 kafka_2.11-0.10.1.

[jira] [Comment Edited] (KAFKA-4277) creating ephemeral node already exist

2017-02-14 Thread Wrikken (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15866415#comment-15866415 ] Wrikken edited comment on KAFKA-4277 at 2/14/17 7:16 PM: - Reproduc

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

2017-02-14 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: don't throw CommitFailedException during suspendTasksAndState -- [...truncated 15979 lines...] org.apache.kafka.common.record.MemoryRecordsTest > testFilterToPres

Re: [VOTE] KIP-82 Add Record Headers

2017-02-14 Thread Onur Karaman
+1 On Tue, Feb 14, 2017 at 10:35 AM, radai wrote: > +1 from me. > > also - a more usable link to the discussion thread: > http://markmail.org/message/x5wlkieexinovsz3 > > On Tue, Feb 14, 2017 at 9:42 AM, Michael Pearce > wrote: > > > Hi all, > > > > We would like to start the voting process for

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-14 Thread Steven Schlansker
Hi, it looks like I have 2 of the 3 minimum votes, can a third voter please consider this KIP? Thanks. (PS - new revision on GitHub PR with hopefully the last round of improvements) > On Feb 8, 2017, at 9:06 PM, Matthias J. Sax wrote: > > +1 > > On 2/8/17 4:51 PM, Gwen Shapira wrote: >> +1 (b

Re: [VOTE] KIP-82 Add Record Headers

2017-02-14 Thread Renu Tewari
+1 after the comprehensive discussion great to see this moving to a vote. On Tue, Feb 14, 2017 at 1:07 PM, Onur Karaman wrote: > +1 > > On Tue, Feb 14, 2017 at 10:35 AM, radai > wrote: > > > +1 from me. > > > > also - a more usable link to the discussion thread: > > http://markmail.org/messa

Re: [VOTE] KIP-82 Add Record Headers

2017-02-14 Thread Jay Kreps
Couple of things I think we still need to work out: 1. I think we agree about the key, but I think we haven't talked about the value yet. I think if our goal is an open ecosystem of these header spread across many plugins from many systems we should consider making this a string as wel

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-14 Thread Jay Kreps
+1 Nice improvement. -Jay On Tue, Feb 14, 2017 at 1:22 PM, Steven Schlansker < sschlans...@opentable.com> wrote: > Hi, it looks like I have 2 of the 3 minimum votes, can a third voter > please consider this KIP? > Thanks. > > (PS - new revision on GitHub PR with hopefully the last round of > im

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

2017-02-14 Thread Apache Jenkins Server
See

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

2017-02-14 Thread Jay Kreps
+1 Super happy with how this turned out. It's been a long journey since we started thinking about this 3+ years ago. Can't wait to see it in code---this is a big one! :-) -Jay On Wed, Feb 1, 2017 at 8:13 PM, Guozhang Wang wrote: > Hi all, > > We would like to start the voting process for KIP-9

[GitHub] kafka pull request #2549: Add subtree .gitignore files themselves to gitigno...

2017-02-14 Thread stevenschlansker
GitHub user stevenschlansker opened a pull request: https://github.com/apache/kafka/pull/2549 Add subtree .gitignore files themselves to gitignore This seems like a silly rule, but for example the Eclipse Gradle integration will 'helpfully' generate .gitignore files in every proj

[GitHub] kafka pull request #2550: KAFKA-4665: Normalize handling of non-existing top...

2017-02-14 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2550 KAFKA-4665: Normalize handling of non-existing topics/partitions in fetching offsets This PR brings some consistency around how non-existing topics or partitions are handled when fetching of

[jira] [Commented] (KAFKA-4665) Inconsistent handling of non-existing topics in offset fetch handling

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

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-14 Thread Zakee
+1 -Zakee > On Feb 14, 2017, at 1:56 PM, Jay Kreps wrote: > > +1 > > Nice improvement. > > -Jay > > On Tue, Feb 14, 2017 at 1:22 PM, Steven Schlansker < > sschlans...@opentable.com> wrote: > >> Hi, it looks like I have 2 of the 3 minimum votes, can a third voter >> please consider this KIP?

[jira] [Updated] (KAFKA-4665) Inconsistent handling of non-existing topics in offset fetch handling

2017-02-14 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4665: --- Status: Patch Available (was: In Progress) > Inconsistent handling of non-existing topics in

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

2017-02-14 Thread Jeyhun Karimov
Dear community, I want to share the KIP-123 [1] which is based on issue KAFKA-4144 [2]. You can check the PR in [3]. I would like to get your comments. [1] https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68714788 [2] https://issues.apache.org/jira/browse/KAFKA-4144 [3] https://g

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

2017-02-14 Thread Mathieu Fenniak
Hi Jeyhun, This KIP might not be the appropriate time, but my first thought reading it is that it might make sense to introduce a builder-style API rather than adding a mix of new method overloads with independent optional parameters. :-) eg. stream(), table(), globalTable(), addSource(), could a

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

2017-02-14 Thread Matthias J. Sax
Mathieu, I personally agree with your observation, and we have plans to submit a KIP like this. If you want to drive this discussion feel free to start the KIP by yourself! Having said that, for this KIP we might want to focus the discussion the the actual feature that gets added: allowing to spe

Re: Correct prefetching of data to KTable-like structure on application startup

2017-02-14 Thread Matthias J. Sax
Jan, If I understand you problem correctly, you do something like this on startup (I simplify to single partition) endOffset = consumer.endOffset(...) while (!done) { for (ConsumerRecord r : consumer.poll()) { // do some processing if (r.offset == endOffset) { done = true;

Re: Is it possible to receive duplicate messages from Kafka when...

2017-02-14 Thread Matthias J. Sax
Yes. That could happen. Kafka provided at-least-once processing semantics if you commit messages after processing. You can avoid duplicates, if you commit offsets before processing, but this might result in data loss. Getting exactly-once is quite hard, and you will need to build your own de-dup

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-14 Thread Jun Rao
Hi, Radai, Currently kafka-acl.sh just creates an ACL path in ZK with the principal name string. The authorizer module in the broker reads the principal name string from the acl path in ZK and creates the expected KafkaPrincipal for matching. As you can see, the expected principal is created on th

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-14 Thread Dong Lin
Hey Jun, Thanks for all your help and time to discuss this KIP. When you get the time, could you let me know if the previous answers address the concern? I think the more interesting question in your last email is where we should store the "created" flag in ZK. I proposed the solution that I like

Re: [VOTE] KIP-82 Add Record Headers

2017-02-14 Thread Michael Pearce
Thanks all so far for the +1 guys. @Jay going to move these questions to the discuss thread and reply there. Please keep an eye out. On 14/02/2017, 21:34, "Renu Tewari" wrote: +1 after the comprehensive discussion great to see this moving to a vote. On Tue, Feb 14, 2017 at 1:07 PM,

[jira] [Commented] (KAFKA-4765) org.apache.kafka.clients.producer.KafkaProducerTest#testConstructorFailureCloseResource and Similar Tests are Failing on some Systems (127.0.53.53 Collision Warning)

2017-02-14 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4765?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15867408#comment-15867408 ] Armin Braun commented on KAFKA-4765: I could add a PR for this if you see it as a vali

[jira] [Created] (KAFKA-4765) org.apache.kafka.clients.producer.KafkaProducerTest#testConstructorFailureCloseResource and Similar Tests are Failing on some Systems (127.0.53.53 Collision Warning)

2017-02-14 Thread Armin Braun (JIRA)
Armin Braun created KAFKA-4765: -- Summary: org.apache.kafka.clients.producer.KafkaProducerTest#testConstructorFailureCloseResource and Similar Tests are Failing on some Systems (127.0.53.53 Collision Warning) Key: KAFKA-4765