Re: Performance issue with KafkaStreams

2016-09-19 Thread Guozhang Wang
Hello Caleb, `./gradlew jar` should be sufficient to run the SimpleBenchmark. Could you look into kafka-run-class.sh and see if "streams/build/libs /kafka-streams*.jar" is added to the dependent path? In trunk it is added. Guozhang On Sat, Sep 17, 2016 at 11:30 AM, Eno Thereska wrote: > Hi C

Re: Performance issue with KafkaStreams

2016-09-17 Thread Eno Thereska
Hi Caleb, I usually do './gradlew installAll' first and that places all the jars in my local maven repo in ~/.m2/repository. Eno > On 17 Sep 2016, at 00:30, Caleb Welton wrote: > > Is there a specific way that I need to build kafka for that to work? > > bash$ export INCLUDE_TEST_JARS=true;

Re: Performance issue with KafkaStreams

2016-09-16 Thread Caleb Welton
Is there a specific way that I need to build kafka for that to work? bash$ export INCLUDE_TEST_JARS=true; ./bin/kafka-run-class.sh org.apache.kafka.streams.perf.SimpleBenchmark Error: Could not find or load main class org.apache.kafka.streams.perf.SimpleBenchmark bash$ find . -name SimpleBenchma

Re: Performance issue with KafkaStreams

2016-09-11 Thread Eno Thereska
Hi Ara, For parallelism you should have as many Kafka Stream instances as partitions. The instances can run on different servers (each instance can have 1 thread). For a single server on the other hand, you can either start multiple single-threaded instances on that server, or a single instance

Re: Performance issue with KafkaStreams

2016-09-10 Thread Ara Ebrahimi
Hi Eno, Could you elaborate more on tuning Kafka Streaming applications? What are the relationships between partitions and num.stream.threads num.consumer.fetchers and other such parameters? On a single node setup with x partitions, what’s the best way to make sure these partitions are consumed

Re: Performance issue with KafkaStreams

2016-09-10 Thread Eno Thereska
Hi Caleb, We have a benchmark that we run nightly to keep track of performance. The numbers we have do indicate that consuming through streams is indeed slower than just a pure consumer, however the performance difference is not as large as you are observing. Would it be possible for you to run

Re: Performance issue with KafkaStreams

2016-09-09 Thread Caleb Welton
Same in both cases: client.id=Test-Prototype application.id=test-prototype group.id=test-consumer-group bootstrap.servers=broker1:9092,broker2:9092zookeeper.connect=zk1:2181 replication.factor=2 auto.offset.reset=earliest On Friday, September 9, 2016 8:48 AM, Eno Thereska wrote: Hi Caleb,

Re: Performance issue with KafkaStreams

2016-09-09 Thread Eno Thereska
Hi Caleb, Could you share your Kafka Streams configuration (i.e., StreamsConfig properties you might have set before the test)? Thanks Eno On Thu, Sep 8, 2016 at 12:46 AM, Caleb Welton wrote: > I have a question with respect to the KafkaStreams API. > > I noticed during my prototyping work tha

Performance issue with KafkaStreams

2016-09-07 Thread Caleb Welton
I have a question with respect to the KafkaStreams API. I noticed during my prototyping work that my KafkaStreams application was not able to keep up with the input on the stream so I dug into it a bit and found that it was spending an inordinate amount of time in org.apache.kafka.common.network.S