consumer throttling based on rate quotas leads to client errors

2016-12-13 Thread Paul Mackles
Hi - We are using kafka_2.11-0.9.0.1. Using the kafka-configs.sh command, we set the consumer_byte_rate for a specific client.id that was misbehaving. The new setting definitely should have led to some throttling. What we found was that connections from that client.id started failing with the

Re: consumer client pause/resume/rebalance

2016-11-08 Thread Paul Mackles
ither. I hope this helps explain the behavior? On Mon, Nov 7, 2016 at 9:53 AM, Paul Mackles wrote: > Using the v0.9.0.1 consumer API, I recently learned that paused partitions > can unexpectedly become become unpaused during a rebalance. I also found an > old thread from the mailing lis

consumer client pause/resume/rebalance

2016-11-07 Thread Paul Mackles
Using the v0.9.0.1 consumer API, I recently learned that paused partitions can unexpectedly become become unpaused during a rebalance. I also found an old thread from the mailing list which corroborates this behavior: http://grokbase.com/t/kafka/users/161wgzckze/new-consumer-pause-reset-behavi

client.id, v9 consumer, metrics, JMX and quotas

2016-05-11 Thread Paul Mackles
Hi I have an app that spins up multiple threads in a single JVM. Each thread has its own v9 consumer running under different groupIds. Since they are part of the same application, I set the client.id property for all 3 consumers to "frylock". Everything runs OK but I do see the following ex

Re: GC is running forever

2015-10-15 Thread Paul Mackles
We had an issue that sounds somewhat similar. It impacted many long-running java apps but Kafka more than most. In our case, it actually turned out to be a kernel bug. Here is a reference with more details: https://groups.google.com/forum/#!topic/mechanical-sympathy/QbmpZxp6C64 Thanks, Paul ___

MetadataRequest vs Zookeeper

2015-02-13 Thread Paul Mackles
I noticed that the standard Kafka storm spout gets topic metadata from zookeeper (under "/brokers/topics/") instead of issuing MetadataRequests to one of the brokers. Aside from possible encapsulation issues, are there any other downsides to using ZK this way? Are there significant cases where Z

Re: How does number of partitions affect sequential disk IO

2014-06-24 Thread Paul Mackles
1 and 3 partitions/node be >negligible? > >> On 24/06/2014, at 9:42 pm, Paul Mackles wrote: >> >> You'll want to account for the number of disks per node. Normally, >> partitions are spread across multiple disks. Even more important, the OS >> file cache red

Re: How does number of partitions affect sequential disk IO

2014-06-24 Thread Paul Mackles
You'll want to account for the number of disks per node. Normally, partitions are spread across multiple disks. Even more important, the OS file cache reduces the amount of seeking provided that you are reading mostly sequentially and your consumers are keeping up. On 6/24/14 3:58 AM, "Daniel Comp

Re: ISR not updating

2014-05-21 Thread Paul Mackles
remain available during the restart. In other words, the replicas were in sync the whole time and it was really just a matter of the ISRs in ZK being out-of-sync. I am not sure if this is an issue in more recent versions. Paul On 5/17/14 9:16 PM, "Paul Mackles" wrote: >Today we

Re: ISR not updating

2014-05-17 Thread Paul Mackles
Rao wrote: > >> Do you see constant ISR shrinking/expansion of those two partitions in >>the >> leader broker's log ? >> >> Thanks, >> >> Jun >> >> >> On Fri, May 16, 2014 at 4:25 PM, Paul Mackles >>wrote: >> >> >

ISR not updating

2014-05-16 Thread Paul Mackles
Hi - We are running kafka_2.8.0-0.8.0-beta1 (we are a little behind in upgrading). >From what I can tell, connectivity to ZK was lost for a brief period. The >cluster seemed to recover OK except that we now have 2 (out of 125) partitions >where the ISR appears to be out of date. In other words,

Re: 0.8 high-level consumer error handling

2014-01-08 Thread Paul Mackles
at would be a suitable action on the application-side if such a >condition were propagated back to the application as an exception? > >Thanks, > >Joel > >On Tue, Jan 07, 2014 at 06:00:29PM +, Paul Mackles wrote: >> Hi - I noticed that if a kafka cluster goes away entirely, the

0.8 high-level consumer error handling

2014-01-07 Thread Paul Mackles
Hi - I noticed that if a kafka cluster goes away entirely, the high-level consumer will endlessly try to fetch metadata until the cluster comes back up, never bubbling the error condition up to the application. While I see a setting to control the interval at which it reconnects, I don't see any

Re: JMXTrans not sending kafka 0.8 metrics to Ganglia

2013-11-05 Thread Paul Mackles
I only see 4 elements as configured above >i.e. > >ReplicaManager.Count >ReplicaManager.OneMinuteRate >ReplicaManager.MeanRate >ReplicaManager.Value > >Then, how can one tell if for e.g. Value pertains to LeaderCount or >PartitionCount? >Even though it is laboriou

Re: JMXTrans not sending kafka 0.8 metrics to Ganglia

2013-11-04 Thread Paul Mackles
It looks like you are missing quotes in the object name. Here is a snippet from our jmxtrans configs: "resultAlias": "ReplicaManager", "obj": "\"kafka.server\":type=\"ReplicaManager\",name=\"*\"", "attr": [ "Count", "OneMinuteRate", "MeanRate", "Value" ] Un

increasing number of replicas for an existing in topic

2013-10-02 Thread Paul Mackles
In 0.8, is there an undocumented way to increase the number of replicas for an existing topic? We created a number of topics with the wrong number of replicas. We could just delete and recreate but delete is flaky in 0.8. Was hoping someone figured out a way to do this w/out deleting the topics.

Re: full disk

2013-09-23 Thread Paul Mackles
Done: https://issues.apache.org/jira/browse/KAFKA-1063 Out of curioisity, is manually removing the older log files the only option at this point? From: Paul Mackles mailto:pmack...@adobe.com>> To: "users@kafka.apache.org<mailto:users@kafka.apache.org>" mailto:users@kafk

broker giving up partition leadership for no apparent reason

2013-09-22 Thread Paul Mackles
With 0.8, we have a situation where a broker is removing itself (or being removed) as a leader for no apparent reason. The cluster has 3 nodes. In this case, broker id=1 stopped leading. This is what I see in the server.log at the time it stopped leading: [2013-09-22 14:00:06,141] INFO re-regis

full disk

2013-09-21 Thread Paul Mackles
Hi - We ran into a situation on our dev cluster (3 nodes, v0.8) where we ran out of disk on one of the nodes . As expected, the broker shut itself down and all of the clients switched over to the other nodes. So far so good. To free up disk space, I reduced log.retention.hours to something more

Re: Unable to send and consume compressed events.

2013-08-29 Thread Paul Mackles
I assume this is kafka 0.8, right? Are there any corresponding errors in the broker logs? With the configuration below, I don't think any errors will be reported back to the producer. You could also try setting erquest.required.acks=1 to see if errors are reported back to the client. On 8/29/13 4

slf4j bindings

2013-08-19 Thread Paul Mackles
Hi – Has anyone figured out a clean way to ignore/exclude the "simple" slf4j bindings that get included in the kafka-assembly jar for 0.8? I would like all of the libaries in my app to log through log4j but for those libraries using slf4j, the "simple" bindings in the kafka-assembly jar are gett

high-level 0.8 consumer API questions

2013-08-16 Thread Paul Mackles
Hi - I am making a few assumptions about the 0.8 high-level consumer API that I am looking to confirm: -it is OK to have multiple ConsumerConnector objects in the same process? To be sure, they are all operating independently. I could probably shove everything into one ConsumerConnector if I ha