ConsoleProducer missing messages (random behavior)

2016-06-16 Thread Radu Radutiu
Hi, I was following the Quickstart guide and I have noticed that ConsoleProducer does not publish all messages (the number of messages published differs from one run to another) and happens mostly on a fresh started broker. version: kafka_2.11-0.10.0.0 OS: Linux (Ubuntu 14.04, Centos 7.2) JDK: jav

Re: About null keys

2016-06-16 Thread Matthias J. Sax
On small addition: If your input topic does not have any key specified, you can still extract a key from the value KTable source = builder.stream(...).selectKey(...).toTable(...) -Matthias On 06/15/2016 08:30 PM, Eno Thereska wrote: > Hi Adrienne, > > How do you enter the input on t1 topic? I

Partition size skew using default partitioner (without key)

2016-06-16 Thread Asaf Mesika
Hi, We've noticed that we have some partitions receiving more messages than others. What I've done to learn that is: * In Kafka Manager, per a given topic, the list of Partition Information is displayed. * For each partition there's a column called Latest Offset - which I assume is the producer of

Re: About null keys

2016-06-16 Thread Adrienne Kole
Thank you for your answers. I was trying to give input to topic from console. Therefore, when I consume the message on kafka streams, I got values with null keys. When I used producer classes of kafka as a Producer, I managed to solve the problem. Thank you for your guidance. Adrienne. On Thu,

Kafka Streams table persistence

2016-06-16 Thread Jan Ehrhardt
Hi, I am curious about the relationship of KTables and state store in Kafka Streams. When I create a table with the `table` method, it uses the original topic as a changelog, thus persisting a table changelog does not make much sense. When I create invoke `countByKey` on a KStream, the result is a

Re: ConsoleProducer missing messages (random behavior)

2016-06-16 Thread Dustin Cote
Hi Radu, Could you clarify what you mean by "I've noticed that if the kafka server was started a long time ago the console producer publishes all lines"? It's a known issue that if there are no brokers available, the producer will drop messages (https://issues.apache.org/jira/browse/KAFKA-1955) an

Groupby Operator

2016-06-16 Thread Davood Rafiei
Hi, I am trying to use groupby operator in simple example. However, I get strange results. I have inputs on "test" topic like: (Long, String) 1Message_1 1Message_1 2Message_2 3Message_3 4Message_4 I want to get counts of each value. So: Message_1 2 Message_1 1 Message_2 1 M

Re: Python kafka client benchmarks

2016-06-16 Thread Michael Noll
Thanks a lot for the investigation and sharing the results, John! -Michael On Thu, Jun 16, 2016 at 7:59 AM, Dana Powers wrote: > Very nice! > > On Wed, Jun 15, 2016 at 6:40 PM, John Dennison > wrote: > > My team has published a post comparing python kafka clients. Might be of > > interest to

Re: Groupby Operator

2016-06-16 Thread Michael Noll
Davood, you are reading the input topic into a KTable, which means that subsequent records for the same key (such as the key `1`, which appears twice in the input messages/records) will be considered as updates to any previous records for that key. So I think what you actually want to do is read

Re: About null keys

2016-06-16 Thread Ian Wrigley
Just fyi, you can specify keys as well as values with kafka-console-producer. Try adding the properties --property parse.key=true --property key.separator=, to your invocation. You can then specify messages as key,value from stdin (For kaka-console-consumer, the related properties are print.key

Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Philippe Derome
The issue had apparently existed and is apparently resolved, but so far it does not work for me: https://issues.apache.org/jira/browse/KAFKA-2203. I issue same command as Stevo Slavic with JDK 1.8.0_91: gradle clean jarAll java -version java version "*1.8.0_91*" Java(TM) SE Runtime Environment

RE: Kafka Sreams Python API

2016-06-16 Thread Ramanan, Buvana (Nokia - US)
Guozhang, Any news on ETA of python client support for Kafka Streams? Thanks, Buvana -Original Message- From: EXT Guozhang Wang [mailto:wangg...@gmail.com] Sent: Thursday, April 21, 2016 4:58 PM To: users@kafka.apache.org Subject: Re: Kafka Sreams Python API We at Confluent are indeed

Re: Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Eno Thereska
Hi Phil, Feel free to comment on that JIRA and re-open if necessary. Eno > On 16 Jun 2016, at 17:02, Philippe Derome wrote: > > The issue had apparently existed and is apparently resolved, but so far it > does not work for me: https://issues.apache.org/jira/browse/KAFKA-2203. > > I issue sam

Re: Kafka Streams table persistence

2016-06-16 Thread Eno Thereska
Hi Jan, It's a good question. Your observations are correct. A KTable is sometimes materialised to a changelog topic lazily, depending on whether the result in the KTable is needed for subsequent operations or not. We are working to improving the documentation of the materialisation rules and

Topic relentlessly recreated

2016-06-16 Thread Jason Kania
We encountered a corrupted topic and when we attempt to delete it, it comes back with some unusable defaults. It's really, really annoying. We are shutting down all the kafka brokers, removing the kafka log folder and contents on all nodes, removing the broker topic information from zookeeper and

Re: Topic relentlessly recreated

2016-06-16 Thread Grant Henke
Hi Jason, We encountered a corrupted topic and when we attempt to delete it, it comes > back with some unusable defaults. It's really, really annoying. > It sounds like you may have auto topic creation enabled and a client is constantly requesting that topic causing it to be created. Try setting

Re: Topic relentlessly recreated

2016-06-16 Thread Jason Kania
Grant, Thanks for the response. We currently have no client running. Does the autocreate happen even if the client is not running? We do have only 2 brokers running so I will recommend an increase for this reason and try that way. We do have delete.topic.enable=true but all that happens is we get

Re: concept of record vs request vs batch

2016-06-16 Thread allen chan
Can anyone help with this question? On Tue, Jun 14, 2016 at 1:45 PM, allen chan wrote: > Thanks for answer Otis. > The producer that i use (Logstash) does not track message sizes. > > I already loaded all the metrics from JMX into my monitoring system. > I just need to confirm that "record" is e

[DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Ismael Juma
Hi all, I would like to start a discussion on making Java 8 a minimum requirement for Kafka's next feature release (let's say Kafka 0.10.1.0 for now). This is the first discussion on the topic so the idea is to understand how people feel about it. If people feel it's too soon, then we can pick up

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Rajiv Kurian
+1 On Thu, Jun 16, 2016 at 1:45 PM, Ismael Juma wrote: > Hi all, > > I would like to start a discussion on making Java 8 a minimum requirement > for Kafka's next feature release (let's say Kafka 0.10.1.0 for now). This > is the first discussion on the topic so the idea is to understand how > peo

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Henry Cai
+1 for Lambda expression. On Thu, Jun 16, 2016 at 1:48 PM, Rajiv Kurian wrote: > +1 > > On Thu, Jun 16, 2016 at 1:45 PM, Ismael Juma wrote: > > > Hi all, > > > > I would like to start a discussion on making Java 8 a minimum requirement > > for Kafka's next feature release (let's say Kafka 0.10.

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Craig Swift
+1 Craig J. Swift Principal Software Engineer - Data Pipeline ReturnPath Inc. Work: 303-999-3220 Cell: 720-560-7038 On Thu, Jun 16, 2016 at 2:50 PM, Henry Cai wrote: > +1 for Lambda expression. > > On Thu, Jun 16, 2016 at 1:48 PM, Rajiv Kurian wrote: > > > +1 > > > > On Thu, Jun 16, 2016 at 1:

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Ben Davison
+ 1 On Thursday, 16 June 2016, Craig Swift wrote: > +1 > > Craig J. Swift > Principal Software Engineer - Data Pipeline > ReturnPath Inc. > Work: 303-999-3220 Cell: 720-560-7038 > > On Thu, Jun 16, 2016 at 2:50 PM, Henry Cai > wrote: > > > +1 for Lambda expression. > > > > On Thu, Jun 16, 2016

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Adam Kunicki
+1 Adam Kunicki StreamSets | Field Engineer mobile: 415.890.DATA (3282) | linkedin On Thu, Jun 16, 2016 at 1:56 PM, Craig Swift < craig.sw...@returnpath.com.i

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread BigData dev
+1. Thanks, Bharat On Thu, Jun 16, 2016 at 1:59 PM, Adam Kunicki wrote: > +1 > > Adam Kunicki > StreamSets | Field Engineer > mobile: 415.890.DATA (3282) | linkedin > < > https://mailtrack.io/trace/link/3e560367e0508b2f285512f39bd070275e70f571?url=http%3A%2F%2Fwww.adamkunicki.com&signature=aab

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Philippe Derome
I strongly support motion having difficulty running (Apache Kafka as opposed to Confluent) Stream examples with JDK 8 today. On 16 Jun 2016 4:46 p.m., "Ismael Juma" wrote: > Hi all, > > I would like to start a discussion on making Java 8 a minimum requirement > for Kafka's next feature release (l

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Stephen Powis
+1 On Thu, Jun 16, 2016 at 5:05 PM, Philippe Derome wrote: > I strongly support motion having difficulty running (Apache Kafka as > opposed to Confluent) Stream examples with JDK 8 today. > On 16 Jun 2016 4:46 p.m., "Ismael Juma" wrote: > > > Hi all, > > > > I would like to start a discussion o

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Jeff Klukas
Would the move to Java 8 be for all modules? I'd have some concern about removing Java 7 compatibility for kafka-clients and for kafka streams (though less so since it's still so new). I don't know how hard it will be to transition a Scala 2.11 application to Scala 2.12. Are we comfortable with the

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Grant Henke
I definitely think its time, but I am not sure if 0.10.1.0 is the release version to drop support in. Would it make sense to drop Java 1.7 and Scala 2.10 support in Kafka 0.11 to maintain upgrade compatibility throughout the 0.10.x series? As a side note, I personally like the pattern of always s

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Stephen Boesch
@Jeff Klukas What is the concern about scala 2.11 vs 2.12? 2.11 runs on both java7 and java8 2016-06-16 14:12 GMT-07:00 Jeff Klukas : > Would the move to Java 8 be for all modules? I'd have some concern about > removing Java 7 compatibility for kafka-clients and for kafka streams > (though less

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Ismael Juma
On Thu, Jun 16, 2016 at 11:13 PM, Stephen Boesch wrote: > @Jeff Klukas What is the concern about scala 2.11 vs 2.12? 2.11 runs on > both java7 and java8 > Scala 2.10.5 and 2.10.6 also support Java 8 for what it's worth. Ismael

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Ismael Juma
Hi Grant, Comments below. On Thu, Jun 16, 2016 at 11:13 PM, Grant Henke wrote: > Would it make sense to drop Java 1.7 and Scala 2.10 support in Kafka 0.11 > to maintain upgrade compatibility throughout the 0.10.x series? > The issue is that we don't know when Kafka 0.11 will be released. Also,

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread jan . omar
Hi Ismael, Unfortunately Java 8 doesn't play nice with FreeBSD. We have seen a lot of JVM crashes running our 0.9 brokers on Java 8... Java 7 on the other hand is totally stable. Until these issues have been addressed, this would cause some serious issues for us. Regards Jan

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Ismael Juma
Hi Jan, That's interesting. Do you have some references you can share on this? It would be good to know which Java 8 versions have been tested and whether it is something that is being worked on. Ismael On Fri, Jun 17, 2016 at 12:02 AM, wrote: > > Hi Ismael, > > Unfortunately Java 8 doesn't pl

Re: Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Philippe Derome
I will. thanks On 16 Jun 2016 1:30 p.m., "Eno Thereska" wrote: > Hi Phil, > > Feel free to comment on that JIRA and re-open if necessary. > > Eno > > > > On 16 Jun 2016, at 17:02, Philippe Derome wrote: > > > > The issue had apparently existed and is apparently resolved, but so far > it > > doe

Re: [DISCUSS] Java 8 as a minimum requirement

2016-06-16 Thread Harsha
-1 on removing suport 0.10.1.0 . This is minor release and removing support JDK 1.7 which lot of users still depend on not a good idea and definitely they are not getting enough heads up to migrate their other services to JDK1.7. We can consider this for 0.11.0 release time line again depends on t

Re: Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Ismael Juma
Hi Philippe, Building with Java 8 works for me. Can you please try `./gradlew clean jarAll` instead? Ismael On Fri, Jun 17, 2016 at 12:23 AM, Philippe Derome wrote: > I will. > > thanks > On 16 Jun 2016 1:30 p.m., "Eno Thereska" wrote: > > > Hi Phil, > > > > Feel free to comment on that JIRA

Kafka Streams KTable-KTable Join Error

2016-06-16 Thread Tim Renner
Hi all, I'm trying to do a KTable-KTable join to compute an average within a tumbling window. Here's the KStreams code (I've put a fully working example in a gist: https://gist.github.com/timothyrenner/a99c86b2d6ed2c22c8703e8c7760af3a) KStreamBuilder builder = new KStreamBuilder(); KStream longs

Re: Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Philippe Derome
I just did and it still fails. Should I use a --info --debug --stacktrace and provide output into JIRA? :core:compileJava UP-TO-DATE :core:compileScala UP-TO-DATE :core:processResources UP-TO-DATE :core:classes UP-TO-DATE :clients:compileTestJava UP-TO-DATE :clients:processTestResources UP-TO

Re: Fail to build examples with gradle on Kafka using JDK 8

2016-06-16 Thread Philippe Derome
Thanks a lot for that. I must had done something wrong. I reinstalled gradle on OSX 10.11 using MacPorts, then recloned kafka and then followed the normal steps and it worked just fine. On Thu, Jun 16, 2016 at 6:49 PM, Ismael Juma wrote: > Hi Philippe, > > Building with Java 8 works for me. Can

Message loss with kafka 0.8.2.2

2016-06-16 Thread Gulia, Vikram
Hi Users, I am facing message loss while using kafka v 0.8.2.2. Please see details below and help me if you can. Issue: 2 messages produced to same partition one by one – Kafka producer returns same offset back which means message produced earlier is lost.

Corrupt Message in 0.8.2.2

2016-06-16 Thread Anishek Agarwal
Hello, As part of the log cleaning which compacts the topic we saw that it encountered the following exception kafka.message.InvalidMessageException: Message found with corrupt size (0) How do we proceed with either configuring log cleaner to skip it or some way for us to remove that message ?

Re: Kafka Sreams Python API

2016-06-16 Thread Guozhang Wang
Buvana, Still no updates on concrete ETA yet. And since our near-term plan is to focus on features such as exactly-once and queryable state support, it may not come out soon in the next couple of months. Guozhang On Thu, Jun 16, 2016 at 8:27 AM, Ramanan, Buvana (Nokia - US) < buvana.rama...@nok

Re: Kafka Streams KTable-KTable Join Error

2016-06-16 Thread Guozhang Wang
Hello Tim, By looking through the source code I suspect it is a bug in Kafka Stream's KStreamWindowReduce implementation. I'll do further investigation tomorrow and possibly file a JIRA with a patch. Guozhang On Thu, Jun 16, 2016 at 4:12 PM, Tim Renner wrote: > Hi all, > > I'm trying to do a

Re: WordCount example errors

2016-06-16 Thread Michael Noll
Jeyhun, just to confirm for you: Kafka Streams only works with Kafka 0.10 brokers [1]. Best, Michael [1] http://docs.confluent.io/3.0.0/streams/faq.html#can-i-use-kafka-streams-with-kafka-clusters-running-0-9-0-8-or-0-7 On Tue, Jun 14, 2016 at 3:03 PM, Jeyhun Karimov wrote: > Thanks for rep

Re: Message loss with kafka 0.8.2.2

2016-06-16 Thread Gerard Klijs
You could try set the acks to -1, so you wait for the produce to be succesfull, until most other brokers also received the message. Another thing you could try is set the unclean.leader.election.enable to false (this is a setting on the broker). I think what's happening now is that the message in y