Re: KafkaProducer "send" blocks on first attempt with Kafka server offline

2016-03-30 Thread Steven Wu
ign. > > > > In any event, there is a configuration that you can tweak to set the max > > time the producer will spend blocking in send(): max.block.ms > > > > -Dana > > > > > >> On Tue, Mar 29, 2016 at 7:26 PM, Steven Wu <stevenz...@gmail.com&

Re: KafkaProducer "send" blocks on first attempt with Kafka server offline

2016-03-29 Thread Steven Wu
I also agree that returning a Future should never block. I have brought this up when 0.8.2 was first released for new Java producer. As Oleg said, KafkaProducer can also block if metadata is not fetched. This is probably more often than offline broker, because metadata is loaded lazily when

Re: some producers stuck when one broker is bad

2015-09-12 Thread Steven Wu
rs solved > the problem and you did not find any unusual things in the logs on brokers > , it is likely that the process was up but was isolated from producer > request and since the producer did not have timeout the producer buffer > filled up. > > Thanks, > > Mayuresh > > > O

Re: some producers stuck when one broker is bad

2015-09-11 Thread Steven Wu
etadataRequest which tells the > producer who are the new leaders for the partitions and the producer > started sending messages to those brokers. > > KAFKA-2120 will handle all of this for you automatically. > > Thanks, > > Mayuresh > > On Tue, Sep 8, 2015 at 8:26 P

some producers stuck when one broker is bad

2015-09-08 Thread Steven Wu
We have observed that some producer instances stopped sending traffic to brokers, because the memory buffer is full. those producers got stuck in this state permanently. Because we couldn't find out which broker is bad here. So I did a rolling restart the all brokers. after the bad broker got

producer metadata behavior when topic not created

2015-06-09 Thread Steven Wu
Hi, I am talking about the 0.8.2 java producer. In our deployment, we disables auto topic creation, because we would like to control the precise number of partitions created for each topic and the placement of partitions (e.g. zone-aware). I did some experimentation and checked the code.

Re: HDD or SSD or EBS for kafka brokers in Amazon EC2

2015-06-03 Thread Steven Wu
that kernel panic issue? Our company will still be running on AMI image 12.04 for a while, I will see whether the fix was also ported onto Ubuntu 12.04 On Tue, Jun 2, 2015 at 2:53 PM, Steven Wu stevenz...@gmail.com wrote: now I remember we had same kernel panic issue in the first week of D2

Re: HDD or SSD or EBS for kafka brokers in Amazon EC2

2015-06-02 Thread Steven Wu
Wes/Daniel, can you elaborate what kind of instability you have encountered? we are on Ubuntu 14.04.2 and haven't encountered any issues so far. in the announcement, they did mention using Ubuntu 14.04 for better disk throughput. not sure whether 14.04 also addresses any instability issue you

Re: HDD or SSD or EBS for kafka brokers in Amazon EC2

2015-06-02 Thread Steven Wu
Nelson daniel.nel...@vungle.com June 2, 2015 at 4:39 PM On Jun 2, 2015, at 1:22 PM, Steven Wu stevenz...@gmail.com stevenz...@gmail.com wrote: can you elaborate what kind of instability you have encountered? We have seen the nodes become completely non-responsive. Usually they get

Re: HDD or SSD or EBS for kafka brokers in Amazon EC2

2015-06-02 Thread Steven Wu
EBS (network attached storage) has got a lot better over the last a few years. we don't quite trust it for kafka workload. At Netflix, we were going with the new d2 instance type (HDD). our perf/load testing shows it satisfy our workload. SSD is better in latency curve but pretty comparable in

Re: Async Producer Callback

2015-03-19 Thread Steven Wu
in your callback impl object, you can save a reference to the actual message. On Wed, Mar 18, 2015 at 10:45 PM, sunil kalva kalva.ka...@gmail.com wrote: Hi How do i access the actual message which is failed to send to cluster using Callback interface and onCompletion method. Basically if

Re: non-blocking sends when cluster is down

2015-02-26 Thread Steven Wu
metadata fetch only happens/blocks for the first time you call send. after the metadata is retrieved can cached in memory. it will not block again. so yes, there is a possibility it can block. of course, if cluster is down and metadata was never fetched, then every send can block. metadata is

Re: New Producer - Is the configurable partitioner gone?

2015-02-22 Thread Steven Wu
The low connection partitioner might work for this by attempting to reuse recently used nodes whenever possible. That is useful in environments with lots and lots of producers where you don't care about semantic partitioning. In one of the perf test, we found that above sticky partitioner

Re: New Producer - Is the configurable partitioner gone?

2015-02-22 Thread Steven Wu
as much an opportunity for improvement in the code as anything. Would you be willing to share the details? -jay On Sunday, February 22, 2015, Steven Wu stevenz...@gmail.com wrote: The low connection partitioner might work for this by attempting to reuse recently used nodes whenever possible

Re: big cpu jump on producer in face of broker outage

2015-02-19 Thread Steven Wu
AM, Steven Wu stevenz...@gmail.com wrote: forgot to mention in case it matters producer: 0.8.2-beta broker: 0.8.1.1 On Thu, Feb 19, 2015 at 10:34 AM, Steven Wu stevenz...@gmail.com wrote: I think this is an issue caused by KAFKA-1788. I was trying to test producer resiliency

big cpu jump on producer in face of broker outage

2015-02-19 Thread Steven Wu
I think this is an issue caused by KAFKA-1788. I was trying to test producer resiliency to broker outage. In this experiment, I shutdown all brokers and see how producer behavior. Here are the observations 1) kafka producer can recover from kafka outage. i.e. send resumed after brokers came back

Re: big cpu jump on producer in face of broker outage

2015-02-19 Thread Steven Wu
forgot to mention in case it matters producer: 0.8.2-beta broker: 0.8.1.1 On Thu, Feb 19, 2015 at 10:34 AM, Steven Wu stevenz...@gmail.com wrote: I think this is an issue caused by KAFKA-1788. I was trying to test producer resiliency to broker outage. In this experiment, I shutdown all

Re: could new java producer miss callbacks after successful send?

2015-02-10 Thread Steven Wu
couldn't reproduce/confirm the issue with my test. send 6 million msgs from 6 instances. got 6 million callbacks. this could be some metric issues. On Mon, Feb 9, 2015 at 8:23 PM, Steven Wu stevenz...@gmail.com wrote: I don't have strong evidence that this is a bug yet. let me write some test

could new java producer miss callbacks after successful send?

2015-02-09 Thread Steven Wu
We observed some small discrepancy in messages sent per second reported at different points. 1) and 4) matches very close. 2) and 3) matches very close but are about *5-6% lower* compared to 1) and 4). 1) send attempt from producer 2) send success from producer 3) record-send-rate reported by

Re: could new java producer miss callbacks after successful send?

2015-02-09 Thread Steven Wu
On Mon, Feb 9, 2015 at 5:19 PM, Steven Wu stevenz...@gmail.com wrote: We observed some small discrepancy in messages sent per second reported at different points. 1) and 4) matches very close. 2) and 3) matches very close but are about *5-6% lower* compared to 1) and 4). 1) send attempt

Re: error handling with high-level consumer

2015-02-05 Thread Steven Wu
if the data is corrupted on disk? Thanks, Jun On Wed, Feb 4, 2015 at 9:55 AM, Steven Wu stevenz...@gmail.com wrote: Hi, We have observed these two exceptions with consumer *iterator.next()* recently. want to ask how should we handle them properly. *1) CRC corruption* Message

high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
Hi, We have observed high cpu and high network traffic problem when 1) cluster (0.8.1.1) has no topic 2) KafkaProducer (0.8.2-beta) object is created without sending any traffic We have observed such problem twice. In both cases, problem went away immediately after one/any topic is created. Is

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
the same test with the 0.8.2. final release? -Jay On Tue, Feb 3, 2015 at 8:37 PM, Steven Wu stevenz...@gmail.com wrote: actually, my local test can reproduce the issue although not immediately. seems to happen after a few mins. I enabled TRACE level logging. here seems to be the tight

Re: high cpu and network traffic when cluster has no topic

2015-02-03 Thread Steven Wu
360186. kafka-producer-network-thread | foo 20:34:32,626 DEBUG NetworkClient:369 - Trying to send metadata request to node -2 On Tue, Feb 3, 2015 at 8:10 PM, Steven Wu stevenz...@gmail.com wrote: Hi, We have observed high cpu and high network traffic problem when 1) cluster (0.8.1.1) has

Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-01 Thread Steven Wu
In Netflix, we have been using route53 DNS name as bootstrap servers in AWS env. Basically, when a kafka broker start, we add it to route53 DNS name for the cluster. this is like the VIP that Jay suggested. But we are also moving toward to use Eureka service registry for bootstrapping. We are

Re: Using Only one partition to store object change log

2015-01-30 Thread Steven Wu
do you need total ordering among all events? or you just need ordering by some partitionKey (e.g. events regarding one particular database key or user id)? if it's the later, you can create multiple partitions and just partition your events using the key to different kafka partitions. On Fri, Jan

Re: [DISCUSSION] Boot dependency in the new producer

2015-01-26 Thread Steven Wu
Jay, I don't think this line will bootstrap full metadata (for all topics). it will just construct the cluster object with bootstrap host. you need to do metadata.add(topic) to set interest of a topic's partition metadata. Guozhang, I personally think this is ok. it just do a few DNS lookup or

Re: Kafka 0.8.2 new producer blocking on metadata

2015-01-05 Thread Steven Wu
problem--doing remote communication during initialization which tends to bite a lot of people. But since this would be an option that would default to false perhaps it would be less likely to come as a surprise. -Jay On Mon, Dec 29, 2014 at 8:38 AM, Steven Wu stevenz...@gmail.com wrote: +1

Re: Kafka 0.8.2 new producer blocking on metadata

2014-12-29 Thread Steven Wu
+1. it should be truly async in all cases. I understand some challenges that Jay listed in the other thread. But we need a solution nonetheless. e.g. can we maintain a separate list/queue/buffer for pending messages without metadata. On Tue, Dec 23, 2014 at 12:57 PM, John Boardman

Re: Is Kafka documentation regarding null key misleading?

2014-12-11 Thread Steven Wu
Guozhang, can you point me to the code that implements periodic/sticky random partitioner? I actually like to try it out in our env, even though I assume it is NOT ported to 0.8.2 java producer. Thanks, Steven On Mon, Dec 8, 2014 at 1:43 PM, Guozhang Wang wangg...@gmail.com wrote: Hi Yury,

Re: [DISCUSSION] adding the serializer api back to the new java producer

2014-12-09 Thread Steven Wu
In practice the cases that actually mix serialization types in a single stream are pretty rare I think just because the consumer then has the problem of guessing how to deserialize, so most of these will end up with at least some marker or schema id or whatever that tells you how to read the

Re: How to produce and consume events in 2 DCs?

2014-10-22 Thread Steven Wu
would mirroring to another cluster. Neha's proposal would work also but I assume its a lot more work for the Kafka internals and therefor IMHO wouldn't meet the kiss principle. Kind regards, Erik. Steven Wu schreef op 22-10-14 om 01:48: I think it doesn't have to be two more clusters. can

Re: How to produce and consume events in 2 DCs?

2014-10-21 Thread Steven Wu
I think it doesn't have to be two more clusters. can be just two more topics. MirrorMaker can copy from source topics in both regions into one aggregate topic. On Tue, Oct 21, 2014 at 1:54 AM, Erik van oosten e.vanoos...@grons.nl.invalid wrote: Thanks Neha, Unfortunately, the maintenance

how to identify rogue consumer

2014-10-08 Thread Steven Wu
I have seen very high Fetch-Consumer-RequestsPerSec (like 15K) per broker in a relatively idle cluster. My hypothesis some misbehaving consumer has a tight polling loop without any back-off logic with empty fetch. Unfortunately, this metric doesn't have per-topic breakdown like BytesInPerSec or

Re: how to identify rogue consumer

2014-10-08 Thread Steven Wu
On Wed, Oct 8, 2014 at 8:57 PM, Steven Wu stevenz...@gmail.com wrote: I have seen very high Fetch-Consumer-RequestsPerSec (like 15K) per broker in a relatively idle cluster. My hypothesis some misbehaving consumer has a tight polling loop without any back-off logic with empty fetch

Re: BytesOutPerSec is more than BytesInPerSec.

2014-09-25 Thread Steven Wu
couldn't see your graph. but your replicator factor is 2. then replication traffic can be the explanation. basically, BytesOut will be 2x of BytesIn. On Thu, Sep 25, 2014 at 6:19 PM, ravi singh rrs120...@gmail.com wrote: I have set up my kafka broker with as single producer and consumer. When I

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-10 Thread Steven Wu
(ZkEventThread.java:71) On Tue, Sep 9, 2014 at 7:18 PM, Steven Wu stevenz...@gmail.com wrote: previous email is from state-change.log. also found this WARN in controller.log [2014-09-09 01:10:53,669] WARN [Controller 5]: Cannot remove replica 0 from ISR of partition [cdn_selection_runtime_data

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
noticed one important thing. topic foo's partition 1 and 2 have empty .log file on replicas. I suspect replication doesn't create the partition dir on broker 0 in this case, which then cause the WARN logs. On Mon, Sep 8, 2014 at 11:11 PM, Steven Wu stevenz...@gmail.com wrote: sorry. forgot

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
of broker 0, do you see anything related to these 2 partitions when broker 0 comes up? Thanks, Jun On Tue, Sep 9, 2014 at 9:41 AM, Steven Wu stevenz...@gmail.com wrote: noticed one important thing. topic foo's partition 1 and 2 have empty .log file on replicas. I suspect replication doesn't

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
broker 0 being detected as the new broker when broker 0 comes up? Thanks, Jun On Tue, Sep 9, 2014 at 3:51 PM, Steven Wu stevenz...@gmail.com wrote: nope. sate-change log files only had some warnings regarding other partitions. nothing related to these two partitions. 2014-09-09 02:54

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
) at kafka.controller.ReplicaStateMachine$BrokerChangeListener.handleChildChange(ReplicaStateMachine.scala:327) at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) On Tue, Sep 9, 2014 at 4:08 PM, Steven Wu stevenz

Re: zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-09 Thread Steven Wu
) On Tue, Sep 9, 2014 at 4:14 PM, Steven Wu stevenz...@gmail.com wrote: ah. maybe you mean the controller log on leader/controller broker 5. yes. I do noticed some errors regarding these two partitions. [2014-09-09 01:10:53,651] ERROR Controller 5 epoch 5 encountered error while changing

new broker instance can't sync up one partition from peers

2014-09-08 Thread Steven Wu
I did a push. new instance comes up and tries to fetch log/data from other peers/replicas. Out of 60 partitions assigned for broker 0, it sync'ed up 59. but for whatever reason, it didn't try to fetch this partition/topic. [out-of-sync replica] BrokerId: 0, Topic: foo, PartitionId: 6, Leader: 5,

zookeeper shows partition in-sync but local disk doesn't have the data dirs

2014-09-08 Thread Steven Wu
did a push in cloud. after new instance for broker 0 comes up, I see a lot of WARNs in log file. 2014-09-09 04:21:09,271 WARN kafka.utils.Logging$class:83 [request-expiration-task] [warn] [KafkaApi-0] Fetch request with correlation id 51893 from client 1409779957450-6014fc32-0-0 on partition

Re: undesirable log retention behavior

2014-07-31 Thread Steven Wu
#brokerconfigs /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop / On Jul 31, 2014 6:52 PM, Steven Wu steve...@netflix.com.invalid wrote

log retention and rollover

2014-06-02 Thread Steven Wu
This might be a bit unusual. We have a topic that we only need to keep last 5 minutes of msgs so that replay from beginning is fast. Although retention.ms has time unit of minute, segment.ms ONLY has time unit of hour. If I understand cleanup correctly, it can only delete files that are rolled

Re: log retention and rollover

2014-06-02 Thread Steven Wu
on hours to avoid too frequent log rolling and in turn too small segment files. For your case this may be reasonable to set the rolling criterion on minutes. Could you file a JIRA? Guozhang On Mon, Jun 2, 2014 at 4:00 PM, Steven Wu steve...@netflix.com.invalid wrote: This might be a bit

Re: kafka broker failed to recovery from ZK failure

2014-05-26 Thread Steven Wu
the actual danger. On Thu, May 22, 2014 at 8:49 PM, Jun Rao jun...@gmail.com wrote: Delete topic is not supported in 0.8.1.1. How did you do it? Thanks, Jun On Thu, May 22, 2014 at 9:59 AM, Steven Wu steve...@netflix.com wrote: yes. I deleted a topic. but not at the time. it's a few hours

Re: kafka broker failed to recovery from ZK failure

2014-05-22 Thread Steven Wu
) at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568) at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) Thanks, Jun On Tue, May 13, 2014 at 9:50 AM, Steven Wu steve...@netflix.com wrote: sorry for the wrong log file. please see the attached zip file for all 4 log files

Re: question about mirror maker

2014-05-12 Thread Steven Wu
if placing mirror maker in the same datacenter as target cluster, it/consumer will talks to zookeeper in remote/source datacenter. would it more susceptible to network problems? As for the problem commit offset without actually producing/writing msgs to target cluster, it can be solved by