Re: Testing broker failover

2016-08-08 Thread Alex Loddengaard
zation.StringSerializer, > metadata.fetch.timeout.ms=500, > key.serializer=org.apache.kafka.common.serialization.StringSerializer} > > > > > On Mon, Aug 8, 2016 at 9:21 AM, Alex Loddengaard > wrote: > > > Hi Alper, can you share your producer config -- the Properties object? We > >

Re: Testing broker failover

2016-08-08 Thread Alex Loddengaard
Hi Alper, can you share your producer config -- the Properties object? We need to learn more to help you understand the behavior you're observing. Thanks, Alex On Fri, Aug 5, 2016 at 7:45 PM, Alper Akture wrote: > I'm using 0.10.0.0 and testing some failover scenarios. For dev, i have > single

Re: Kafka cluster with a different version that the java API

2016-08-05 Thread Alex Loddengaard
Hi Sergio, clients have to be the same version or older than the brokers. A newer client won't work with an older broker. Alex On Fri, Aug 5, 2016 at 7:37 AM, Sergio Gonzalez < sgonza...@cecropiasolutions.com> wrote: > Hi users, > > Is there some issue if I create the kafka cluster using the > k

Re: log.retention.bytes

2016-06-24 Thread Alex Loddengaard
Hi Dave, log.retention.bytes is per partition. If you change it after the topic was created, you'll see the behavior you expect -- namely that the new value is used when the log is cleaned. The frequency that the log is cleaned is controlled by log.retention.check.interval.ms, with a default value

Re: How to delete data from topic with interval of two days

2016-06-22 Thread Alex Loddengaard
Hi Kotesh, log.retention.hours sets how long messages are kept in the long, and log.retention.check.interval.ms sets how often the log cleaner checks if messages should be deleted based on the retention setting. I hope this helps. Alex On Wed, Jun 22, 2016 at 3:13 AM, kotesh banoth wrote: > H

Re: Question about heterogeneous brokers in a cluster

2016-06-09 Thread Alex Loddengaard
Hi Kevin, If you keep the same configs on the new brokers with more storage capacity, I don't foresee any issues. Although I haven't tried it myself. What may introduce headaches is if you have different configuration options per broker. Or if you try to assign more partitions to the newer broker

Re: Use Zookeeper for a Producer

2016-05-31 Thread Alex Loddengaard
gt; What if number of brokers change? Does it mean I need to change > configuration or potentially recompile my producer and consumer? > > On Tue, May 31, 2016 at 3:27 PM, Alex Loddengaard > wrote: > > > The "old" consumer used ZooKeeper. The "new" consumer,

Re: Use Zookeeper for a Producer

2016-05-31 Thread Alex Loddengaard
The "old" consumer used ZooKeeper. The "new" consumer, introduced in 0.9, doesn't use ZooKeeper. The producer doesn't use ZooKeeper, either. However, brokers still use ZooKeeper. Alex On Tue, May 31, 2016 at 12:03 PM, Igor Kravzov wrote: > When I look at code samples producers mostly write to b

Re: Topics, partitions and keys

2016-05-31 Thread Alex Loddengaard
Hi Igor, see inline: On Sat, May 28, 2016 at 8:14 AM, Igor Kravzov wrote: > I need some clarification on subject. > In Kafka documentations I found the following: > > Kafka only provides a total order over messages *within* a partition, not > between different partitions in a topic. Per-partitio

Re: Kafka error: FATAL Fatal error during KafkaServerStartable startup.after enabling security

2016-05-31 Thread Alex Loddengaard
g.apache.zookeeper.ClientCnxn) > [2016-05-27 23:04:10,295] FATAL Fatal error during KafkaServer startup. > Prepare to shutdown (kafka.server.KafkaServer) > org.I0Itec.zkclient.exception.ZkTimeoutException: Unable to connect to > zookeeper server within timeout: 3 > at org.I0Itec.zkclient.ZkClient.con

Re: Kafka error: FATAL Fatal error during KafkaServerStartable startup.after enabling security

2016-05-27 Thread Alex Loddengaard
Hi Kiran, Can you attach your configuration files not in a .zip? Most likely your broker isn't using the correct hostname:port to connect to ZooKeeper. Although if you're using ZooKeeper SASL, you may have a SASL misconfiguration. Set the `sun.security.krb5.debug` JVM property to `true` to get SA

Re: Separating internal and external traffic

2016-05-25 Thread Alex Loddengaard
Hi there, You can use the `listeners` config to tell Kafka which interfaces to listen on. The `listeners` config also supports setting the port and protocol. You may also want to set `advertised.listeners` if the `listeners` hostnames or IPs aren't reachable by your clients. Alex On Wed, May 25,

Re: Best monitoring tool for Kafka in production

2016-05-25 Thread Alex Loddengaard
Hi Hafsa, We often see Grafana and Graphite, which are both free. Keep in mind you should monitor the system's metrics and Kafka's JMX metrics. Alex On Wed, May 25, 2016 at 3:42 AM, Hafsa Asif wrote: > Hello, > > What is the best monitoring tool for Kafka in production, preferable free > tool?

Re: ISR shrinking/expanding problem

2016-05-16 Thread Alex Loddengaard
Hi Russ, They should eventually catch back up and rejoin the ISR. Did they not? Alex On Fri, May 13, 2016 at 6:33 PM, Russ Lavoie wrote: > Hello, > > I moved an entire topic from one set of brokers to another set of brokers. > The network throughput was so high, that they fell behind the leade

Re: New broker as follower for a topic replication

2016-05-10 Thread Alex Loddengaard
Hi Paolo, The best way to do this would be to have broker3 start up with the same broker id as the failed broker2. broker3 will then rejoin the cluster, begin catching up with broker1, and eventually rejoin the ISR. If it starts up with a new broker id, you'll need to run the partition reassignmen

Re: Consumer stopped after reading some messages

2016-05-10 Thread Alex Loddengaard
Hi Sahitya, I wonder if your consumers are experiencing soft failures because they're busy processing a large collection of messages and not calling poll() within session.timeout.ms? In this scenario, the group coordinator (a broker) would not receive a heartbeat within session.timeout.ms and woul

Re: Backing up Kafka data and using it later?

2016-05-10 Thread Alex Loddengaard
You may find this interesting, although I don't believe it's exactly what you're looking for: https://github.com/pinterest/secor I'm not sure how stable and commonly used it is. Additionally, I see a lot of users use MirrorMaker for a "backup," where MirrorMaker copies all topics from one Kafka

Re: Multiple topics to one consumer

2016-03-08 Thread Alex Loddengaard
Hi there, One consumer can indeed consume from multiple topics (and multiple partitions). For example, see here: http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.List) Then, in your poll() loop, you can get the topic and partition from

Re: Enforce Unique consumer group id in Kafka

2016-02-26 Thread Alex Loddengaard
Hi Nishant, You could use SASL authentication and authorization (ACLs) to control access to topics. In your use case, you would require authentication and control which principals have access to which consumer groups. These features are available in 0.9 but not 0.8. Here are some resources: http:

Re: Creating new consumers after data has been discarded

2016-02-24 Thread Alex Loddengaard
Hi Ted, The largest a partition can get is the size of the disk storing the partition's log on the broker. You can use RAID to increase the "disk" size, and hence the partition size. Whether or not you can fit all messages in Kafka depends on your use case -- how much data you have, the size of yo

Re: Does kafka.common.QueueFullException indicate back pressure in Kafka?

2016-02-22 Thread Alex Loddengaard
nd, > after about 45 minutes or so, I started seeing QueueFullExceptions. This > is why I am wondering if these exceptions can also be a symptom of back > pressure from Kafka. > > Is this plausible? > > --John > > > > I have tuned the producers > > On Thu, Feb 18, 2016 at 3:59

Re: Does kafka.common.QueueFullException indicate back pressure in Kafka?

2016-02-18 Thread Alex Loddengaard
> As always, any thoughts from people who know more than I do are > appreciated. :) > > Thanks > > --John > -- *Alex Loddengaard | **Solutions Architect | Confluent* *Download Apache Kafka and Confluent Platform: www.confluent.io/download <http://www.confluent.io/download>*

Re: kafka.common.QueueFullException

2016-02-18 Thread Alex Loddengaard
y going to increase with time as we > scale. What is the right way to solve this problem ? > > Thanks, > Saurabh > -- *Alex Loddengaard | **Solutions Architect | Confluent* *Download Apache Kafka and Confluent Platform: www.confluent.io/download <http://www.confluent.io/download>*

Re: Consumer seek on 0.9.0 API

2016-02-18 Thread Alex Loddengaard
> anyone can explain me. > > Regards, > > Robin > > 2016-02-17 20:39 GMT+01:00 Alex Loddengaard : > > > Hi Robin, > > > > I believe seek() needs to be called after the consumer gets its partition > > assignments. Try calling poll() before you call seek

Re: Replication Factor and number of brokers

2016-02-17 Thread Alex Loddengaard
Thanks, Alexis and Ben! Alex On Wed, Feb 17, 2016 at 5:57 AM, Ben Stopford wrote: > If you create a topic with more replicas than brokers it should throw an > error but if you lose a broker you'd have under replicated partitions. > > B > > On Tuesday, 16 February 2016,

Re: Consumer seek on 0.9.0 API

2016-02-17 Thread Alex Loddengaard
> > > But this didn't nothing (when I don't use seek() method I consume all the > messages without any problems). > > Any help on this will be greatly appreciated ! > > Regards, > > Robin > -- *Alex Loddengaard | **Solutions Architect | Confluent* *Download Apache Kafka and Confluent Platform: www.confluent.io/download <http://www.confluent.io/download>*

Re: Replication Factor and number of brokers

2016-02-16 Thread Alex Loddengaard
; > > > >HTH, > > >Damian > > > > > >On 16 February 2016 at 14:08, Sean Morris (semorris) < > semor...@cisco.com> > > >wrote: > > > > > >> Should your number of brokers be atleast one more then your > replication >

Re: Encryption on disk

2016-01-15 Thread Alex Loddengaard
red by Kafka is not nearly as > >>> efficient anymore after encrypting the data. Doing this message by > >>> message encryption gives us a performance penalty of about 75%, even if > >>> we compress the messages before encryption. > >>> > >>>

Re: Kafka Connect usage

2016-01-12 Thread Alex Loddengaard
.connect.runtime.WorkerSourceTask:237) > [2016-01-12 11:43:51,948] ERROR Failed to commit offsets for > WorkerSourceTask{id=local-file-source-0} > (org.apache.kafka.connect.runtime.SourceTaskOffsetCommitter:112) > > Is any other configuration required? > > Thanks, > Shiti >

Re: Leader Election

2016-01-06 Thread Alex Loddengaard
copying of this message is strictly prohibited. If you have received this > message in error, please notify the sender immediately and delete all > copies. > -- *Alex Loddengaard | **Solutions Architect | Confluent* *Download Apache Kafka and Confluent Platform: www.confluent.io/download <http://www.confluent.io/download>*

Re: Programmable API for Kafka Connect ?

2016-01-06 Thread Alex Loddengaard
dd/remove Connectors? > > I also do not want to use the REST API and do not want to start the REST > server. > > Thanks, > Shiti > -- *Alex Loddengaard | **Solutions Architect | Confluent* *Download Apache Kafka and Confluent Platform: www.confluent.io/download <http://www.confluent.io/download>*

Re: Topic Deletion Issues

2016-01-05 Thread Alex Loddengaard
Hi Brenden, I sent the wrong permalink. Try this: http://mail-archives.apache.org/mod_mbox/kafka-users/201512.mbox/%3CCAH7vnfhUQHQcCJ1R3pS_DYn0LvmA8fJtLtvCuvvDboWSqm-NBg%40mail.gmail.com%3E Alex On Tue, Jan 5, 2016 at 10:21 AM, Alex Loddengaard wrote: > Hi Brenden, your previous email w

Re: Topic Deletion Issues

2016-01-05 Thread Alex Loddengaard
keeper > node isn't "fully" created as Kafka deletion seems stuck without the > expected node path. > > We are using Kafka 8.2. Purging is also an option (if achievable > programmatically), we just need to make sure there are no messages > left when resuming the pr

Re: Find current kafka memory usage

2016-01-05 Thread Alex Loddengaard
Murphey wrote: > I was running out of heap space for my kafka broker. Is there a way I can > check how much memory kafka is using so I can alert myself if it is > reaching the max heap size? Default is 1GB. > > Thanks > -- *Alex Loddengaard | **Solutions Architect | Confluent

Re: Producing when broker goes down

2015-12-18 Thread Alex Loddengaard
Hi Buck, What are your settings for: - acks - request.timeout.ms - timeout.ms - min.insync.replicas (on the broker) Thanks, Alex On Fri, Dec 18, 2015 at 1:23 PM, Buck Tandyco wrote: > I'm stress testing my kafka setup. I have a producer that is working just > fine and then I kill