Re: Reg. Kafka transactional producer and consumer

2017-11-08 Thread Apurva Mehta
Hi, Your log segment dump and the producer log don't correlate. The producer log shows the producerId == 4001. But your log segment dumps don't have this producerId. Please share data from the same run where you reproduce this issue. For the producerId's 0-4 (shown in the dump), there seem to be

Re: Replication does not start because of OuOfOrderSequenceException

2017-10-02 Thread Apurva Mehta
Hi Stas, Thanks for reporting this. It would be helpful to have JIRA with more of the server logs on the leaders and followers in the time leading up to this OutOfOrderSequenceException. The answers to the following questions would help, when you file the JIRA: What are the retention settings fo

Re: implementing kafka transactions : performance issue

2017-09-18 Thread Apurva Mehta
Hi Hugues. How 'big' are your transactions? In particular, how many produce records are in a single transaction? Can you share your actual producer code? Also, did you try the `kafka-producer-perf-test.sh` tool with a transactional id and see what the latency is for transactions with that tool?

Re: increased response time for OffsetCommit requests

2017-08-03 Thread Apurva Mehta
ease is from 470 ms to around 1.004 s. >- The average batch size (batch-size-avg) is around 320B. >- The linger time is 10ms. > > However, the 99th percentile for OffsetCommit has increased from 1.08 to > 2.8 seconds. > > Best Regards, > Gaurav Abbi > > On Tue, A

Re: Kafka 0.11.0 problem with transactions.

2017-08-03 Thread Apurva Mehta
Ismael raises good questions about what transactions would mean for the console producer. However, the kafka-producer-perf-test script has transactions enabled. It enables you to generate transactions of a certain duration (like 50ms, 100ms). It produces messages of specified size and commits them

Re: increased response time for OffsetCommit requests

2017-08-01 Thread Apurva Mehta
to mention, we also upgraded to 0.11.0.0 client > libraries. > > > We are currently using old Producer and consumer APIs. > > > > > > > > > > > > Best Regards, > > > Gaurav Abbi > > > > > > On Mon, Jul 31, 2017 at 7:46 PM, Apur

Re: increased response time for OffsetCommit requests

2017-07-31 Thread Apurva Mehta
or > rate and a decrease in the responses received per second. > > > One more thing to mention, we also upgraded to 0.11.0.0 client libraries. > We are currently using old Producer and consumer APIs. > > > > Best Regards, > Gaurav Abbi > > On Mon, Jul 31, 2017 at 7:

Re: increased response time for OffsetCommit requests

2017-07-31 Thread Apurva Mehta
How much is the increase? Is there any increase in throughput? On Mon, Jul 31, 2017 at 8:04 AM, Gaurav Abbi wrote: > Hi All, > We recently upgraded to Kafka 0.11.0.0 from 0.10.1.1. > Since then we have been observing increased latencies especially > OffsetCommit requests. > Looking at the server

Re: Consumer throughput drop

2017-07-20 Thread Apurva Mehta
Hi Ovidu, The see-saw behavior is inevitable with linux when you have concurrent reads and writes. However, tuning the following two settings may help achieve more stable performance (from Jay's link): > *dirty_ratio*Defines a percentage value. Writeout of dirty data begins > (via *pdflush*) whe

Re: [VOTE] 0.11.0.0 RC1

2017-06-21 Thread Apurva Mehta
Hi Tom, I actually made modifications to the produce performance tool to do real transactions earlier this week as part of our benchmarking (results published here: bit.ly/kafka-eos-perf). I just submitted that patch here: https://github.com/apache/kafka/pull/3400/files I think my version is more

Re: NPE on startup with a low-level API based application

2017-06-15 Thread Apurva Mehta
Finally, was compression enabled when you hit this exception? If so, which compression algorithm was enabled? On Thu, Jun 15, 2017 at 5:04 PM, Apurva Mehta wrote: > Frank: it would be even better if you could share the key and value which > was causing this problem. Maybe share it on th

Re: NPE on startup with a low-level API based application

2017-06-15 Thread Apurva Mehta
Frank: it would be even better if you could share the key and value which was causing this problem. Maybe share it on the JIRA: https://issues.apache.org/jira/browse/KAFKA-5456 ? Thanks, Apurva On Thu, Jun 15, 2017 at 4:07 PM, Apurva Mehta wrote: > Hi Frank, > > What is is the

Re: NPE on startup with a low-level API based application

2017-06-15 Thread Apurva Mehta
Hi Frank, What is is the value of `batch.size` in your producer? What is the size of the key and value you are trying to write? Thanks, Apurva On Thu, Jun 15, 2017 at 2:28 AM, Frank Lyaruu wrote: > Hey people, I see an error I haven't seen before. It is on a lowlevel-API > based streams applic

Re: [ANNOUNCE] New committer: Rajini Sivaram

2017-04-24 Thread Apurva Mehta
Congratulations Rajini! On Mon, Apr 24, 2017 at 2:06 PM, Gwen Shapira wrote: > The PMC for Apache Kafka has invited Rajini Sivaram as a committer and we > are pleased to announce that she has accepted! > > Rajini contributed 83 patches, 8 KIPs (all security and quota > improvements) and a signif

Re: [DISCUSS] KIP-118: Drop Support for Java 7 in Kafka 0.11

2017-02-03 Thread Apurva Mehta
Thanks Ismael, this makes sense. On Fri, Feb 3, 2017 at 11:50 AM, Guozhang Wang wrote: > LGTM too. > > On Fri, Feb 3, 2017 at 10:39 AM, Eno Thereska > wrote: > > > Makes sense. > > > > Eno > > > > > On 3 Feb 2017, at 10:38, Ismael Juma wrote: > > > > > > Hi all, > > > > > > I have posted a KIP

Re: Messages are lost

2017-01-23 Thread Apurva Mehta
What version of kafka have you deployed? Can you post a thread dump of the hung broker? On Fri, Jan 20, 2017 at 12:14 PM, Ghosh, Achintya (Contractor) < achintya_gh...@comcast.com> wrote: > Hi there, > > I see the below exception in one of my node's log( cluster with 3 nodes) > and then the node

Re: Leader imbalance issue

2017-01-20 Thread Apurva Mehta
Hi Meghana, Have you tried using the 'kafka-prefered-replica-election.sh' script? It will try to move leaders back to the preferred replicas when there is a leader imbalance. https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-1.PreferredReplicaLeaderElectionTool

Re: [VOTE] Vote for KIP-101 - Leader Epochs

2017-01-04 Thread Apurva Mehta
Looks good to me! +1 (non-binding) On Wed, Jan 4, 2017 at 9:24 AM, Ben Stopford wrote: > Hi All > > We’re having some problems with this thread being subsumed by the > [Discuss] thread. Hopefully this one will appear distinct. If you see more > than one, please use this one. > > KIP-101 should

Re: Connectivity problem with controller breaks cluster

2016-12-27 Thread Apurva Mehta
Looks like you are hitting: https://issues.apache.org/jira/browse/KAFKA-4477 You can try upgrading to 0.10.1.1 and see if the issue recurs (a bunch of deadlock bugs were fixed which might explain this issue). Or you can try to provide the data described in https://issues.apache.org/jira/browse/KAF

Re: Kafka Errors and Hung Brokers

2016-12-14 Thread Apurva Mehta
Regarding 1), you can see a NotLeaderForPartition exception if the leader for the partition has moved to another host but the client metadata has not updated itself yet. The messages should disappear once the metadata is updated on all clients. Leaders may move if brokers are bounced, or if they h

Re: lag for a specific partitions on newly added host

2016-12-14 Thread Apurva Mehta
ttempt another reassign? > > Thanks > -jeremy > > > On Dec 13, 2016, at 5:43 PM, Apurva Mehta wrote: > > > > How did you add the host and when did you measure the lag? If you used > the > > reassign-partitions script, it will move partitions to the new host, but >

Re: Reasonable time to commit offsets?

2016-12-14 Thread Apurva Mehta
Hi Gwilym, What is the latency for synchronously producing to this cluster? Is it also 1000 to 2000ms? Thanks, Apurva On Wed, Dec 14, 2016 at 2:17 AM, Gwilym Evans wrote: > Hi folks, > > New to the list and new to operating Kafka. I'm trying to find out what a > reasonable turnaround time for

Re: kafka_2.11-0.9.0.1 crash with java coredump

2016-12-14 Thread Apurva Mehta
I would suggest creating a JIRA and describing in detail what was going on in the cluster when this happened, and posting the associated broker / state change / controller logs. Thanks, Apurva On Wed, Dec 14, 2016 at 3:28 AM, Mazhar Shaikh wrote: > Hi All, > > I am using kafka_2.11-0.9.0.1 with

Re: lag for a specific partitions on newly added host

2016-12-13 Thread Apurva Mehta
How did you add the host and when did you measure the lag? If you used the reassign-partitions script, it will move partitions to the new host, but the data copy will take time. in that period, those partitions will lag. However, once the reassign-partitions script finishes, the partitions on the n

Re: kafka commands taking a long time

2016-12-13 Thread Apurva Mehta
That is certainly odd. What's the latency when using the kafka console producers and consumers? Is it much faster? If it is, I would just strace the kafka-topics command to see where it is spending the time. On Thu, Dec 8, 2016 at 7:21 AM, Stephen Cresswell < stephen.cressw...@gmail.com> wrote: >

Re: failed to delete kafka topic when building from source

2016-12-13 Thread Apurva Mehta
How are you trying to delete the topic? Next time this occurs, can you check whether the process has permissions to perform that operation? On Mon, Dec 12, 2016 at 10:55 PM, Sachin Mittal wrote: > Hi, > I recently built an application from source and I get the following > exception when trying t

Re: ActiveControllerCount is always will be either 0 or 1 in 3 nodes kafka cluster?

2016-12-13 Thread Apurva Mehta
Thanks Sven, I will followup and ensure that the document is tightened up. Apurva On Mon, Dec 12, 2016 at 4:57 AM, Sven Ludwig wrote: > Hi, > > in JMX each Kafka broker has a value 1 or 0 for ActiveControllerCount. As > I understood from this thread, the sum of these values across the cluster >

Re: NotLeaderForPartitionException

2016-12-06 Thread Apurva Mehta
Hi Sven, You will see this exception during leader election. When the leader for a partition moves to another broker, there is a period during which the replicas would still connect to the original leader, at which point they will raise this exception. This should be a very short period, after whi

Re: Is CreateTopics and DeleteTopics ready for production usage?

2016-12-05 Thread Apurva Mehta
I should clarify, that those requests may work, but are not used in any active code. The integration with the rest of the system is yet to happen. On Mon, Dec 5, 2016 at 1:45 PM, Apurva Mehta wrote: > It isn't ready yet. It is part of the work related to > https://cwiki.apache.org

Re: Is CreateTopics and DeleteTopics ready for production usage?

2016-12-05 Thread Apurva Mehta
It isn't ready yet. It is part of the work related to https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations Thanks, Apurva On Mon, Dec 5, 2016 at 11:10 AM, Dmitry Lazurkin wrote: > Hello. > > Are requests CreateTopics and DeleteTopics r

Re: Suggestions

2016-12-02 Thread Apurva Mehta
> > then, the strange thing is that the consumer on > the second topic which stay in poll forever, *without receive any message*. How long is 'forever'? Did you wait more than 5 minutes? On Fri, Dec 2, 2016 at 2:55 AM, Vincenzo D'Amore wrote: > Hi Kafka Gurus :) > > I'm creating process betwe

Re: Tracking when a batch of messages has arrived?

2016-12-02 Thread Apurva Mehta
That should work, though it sounds like you may be interested in : https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging If you can include the 'batch_id' inside your messages, and define custom control messages with a control topic, then you

Re: About stopping a leader

2016-12-01 Thread Apurva Mehta
Yes, the leader should move to K2 or K3. You can check the controller log on all 3 machines to find out where the new leader is placed. It is not guaranteed to move back to K1 when you restart it 2 hours later, however. On Mon, Nov 21, 2016 at 3:38 AM, marcel bichon wrote: > Hello ! > > I have a

Re: Message order different each time stream is replayed?

2016-11-30 Thread Apurva Mehta
How many partitions do you have in that topic. Kafka only guarantees a total ordering of messages within a partition, not across partitions of a topic. If you want total ordering over the entire topic, you need to create a topic with a single partition. On Wed, Nov 30, 2016 at 4:10 AM, Ali Akhtar

Re: Disadvantages of Upgrading Kafka server without upgrading client libraries?

2016-11-29 Thread Apurva Mehta
I may be wrong, but since there have been message format changes between 0.8.2 and 0.10.1, there will be a performance penalty if the clients are not also upgraded. This is because you lose the zero-copy semantics on the server side as the messages have to be converted to the old format before bein

Re: addition

2016-11-21 Thread Apurva Mehta
It's self serve: http://kafka.apache.org/contact On Mon, Nov 21, 2016 at 1:20 AM, marcel bichon wrote: > request of addition to the mailing list >