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

2016-05-11 Thread Manikumar Reddy
Hi, This is known issue. Check below links for related discussion https://issues.apache.org/jira/browse/KAFKA-3494 https://qnalist.com/questions/6420696/discuss-mbeans-overwritten-with-identical-clients-on-a-single-jvm Manikumar On Wed, May 11, 2016 at 7:29 PM, Paul Mackles

Kafka Connect: fork process from a SinkTask ?

2016-05-11 Thread Dean Arnold
I need to run an external filter program from a SinkTask. Is there anything that might break if I fork/exec in the start() method, and forward the data thru pipes ? TIA, Dean

Re: Kafka Connect tasks consumers issue

2016-05-11 Thread Liquan Pei
Hi Matteo, I am not completely follow the steps. Can you share the exact command to reproduce the issue? What kind of commands did you use to restart the connector? Which version of Kafka are you using? Thanks, Liquan On Wed, May 11, 2016 at 4:40 AM, Matteo Luzzi

Migrating Kafka from old VMs to new VMs in a different Cluster

2016-05-11 Thread Abhinav Damarapati
Hello Everyone, We have Kafka brokers, Zookeepers and Mirror-makers running on old Virtual Machines. We need to migrate all of this to brand new VMs on a different DataCenter and bring down the old VMs. Is this possible? If so, please suggest a way to do it. Best, Abhinav

Re: [DISCUSS] KIP-59 - Proposal for a kafka broker command - kafka-brokers.sh

2016-05-11 Thread Gwen Shapira
Hello Jayesh, Thank you for the suggestion. I like the proposal and the new tool seems useful. Do you already have the tool available in a github repository? If you don't, then this would be a good place to start - there are many Kafka utilities in github repositories (Yahoo's Kafka Manager as

Re: Consumer group rebalance taking too long

2016-05-11 Thread tao xiao
Hi Spico, Yes your theory is correct. The sloppy consumer waited in onPartitionRevoked until session timed out and another round of group rebalance was triggered which resulted in Consumer B taking all partitions. After waking up from sleep group rebalance was triggered again On Wed, 11 May 2016

Rolling upgrade from 0.8.2.1 to 0.9.0.1 failing with replicafetchthread OOM errors

2016-05-11 Thread Russ Lavoie
Good Afternoon, I am currently trying to do a rolling upgrade from Kafka 0.8.2.1 to 0.9.0.1 and am running into a problem when starting 0.9.0.1 with the protocol version 0.8.2.1 set in the server.properties. Here is my current Kafka topic setup, data retention and hardware used: 3 Zookeeper

Error reading field 'brokers'

2016-05-11 Thread 0xAB AB
Hi, I am using kafka version e20eba958d8de29cb4e3b6feea37ac3a1e1ab4f9 (something which identifies as 0.10.0-SNAPSHOT when built), which gives me files like ./repository/org/apache/kafka/kafka-tools/0.10.1.0-SNAPSHOT/kafka-tools-0.10.1.0-SNAPSHOT.jar and

RE: New Consumer APIs , Hight level API , Simple API regarding offset management

2016-05-11 Thread Natarajan, Rajeswari
We have a requirement that consumer must be able to re-read the messages. In High level consumer api,it looks like the re-start of consumer needed ,if offset has to be reset. The new consumer API seems to be of beta quality. In stable version ,I guess the only option is to go for simple

Re: Failing between mirrored clusters

2016-05-11 Thread Gerard Klijs
I don't think it's possible since the offsets of both clusters can be different, you don't know if it will work correctly. When I used the mirror maker accidentally on the __consumer_offsets topic it also gave some errors, so I don't know if it's technically possible. A possible future solution

Failing between mirrored clusters

2016-05-11 Thread Ben Stopford
Hi I’m looking at failing-over from one cluster to another, connected via mirror maker, where the __consumer_offsets topic is also mirrored. In theory this should allow consumers to be restarted to point at the secondary cluster such that they resume from the same offset they reached in the

Re: Consumer group rebalance taking too long

2016-05-11 Thread Spico Florin
Hi! Here is a great article about the consumer API: http://www.confluent.io/blog/tutorial-getting-started-with-the-new-apache-kafka-0.9-consumer-client . In my opinion, the consumer will not be able to send the heartbeat to the group coordinator (due to the fact that poll calls on

Re: Kafka 9 version offset storage mechanism changes

2016-05-11 Thread Dustin Cote
Ah, yes those consumers znodes in Zookeeper are related to consumer group metadata. You can actually find a full write up of what each znode is used for here: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper Cheers, On Wed, May 11, 2016 at 9:22 AM, Spico

Re: Kafka 9 version offset storage mechanism changes

2016-05-11 Thread Spico Florin
Hi, Dustin! Thank you for your answer. I have observed that in zookeeper there was a folder named consumers, that kept data about the topic name its partitions and the offsets. Were both the consumers/producers using this folder to keep track of the offsets? What was the purpose of this folder? I

Re: Kafka 9 version offset storage mechanism changes

2016-05-11 Thread Dustin Cote
Hi Florin, The new consumer is intended to replace both the high level and simple consumers. http://kafka.apache.org/documentation.html#consumerapi The old simple consumer API didn't rely on zookeeper to store offsets, but rather the client was responsible for managing their own offsets as

Re: ConsumerGroupCommand in Kafka 0.8.2

2016-05-11 Thread Dustin Cote
Hey Joe, The closest thing is probably the ConsumerOffsetChecker (which is deprecated in 0.9 and replaced by the ConsumerGroupCommand). https://cwiki.apache.org/confluence/display/KAFKA/System+Tools#SystemTools-ConsumerOffsetChecker On Wed, May 11, 2016 at 5:09 AM, Joe San

Re: Can we delete topic in kafka

2016-05-11 Thread Jaikiran Pai
That's actually not the right way to delete topics (or for that matter managing a Kafka instance). It can lead to odd/corrupt installation. -Jaikiran On Wednesday 11 May 2016 06:27 PM, Eduardo Costa Alfaia wrote: Hi, It’s better creating a script that delete the kafka folder where exist the

Re: Can we delete topic in kafka

2016-05-11 Thread Eduardo Costa Alfaia
Hi, It’s better creating a script that delete the kafka folder where exist the kafka topic and after create it again if need. BR Eduardo Costa Alfaia Ph.D. Student in Telecommunications Engineering Università degli Studi di Brescia Tel: +39 3209333018 On 5/11/16, 09:48, "Snehalata

Bug Kafka 0.9.1:StaleMetadataException and consumer thread hang up when performed position method on an unexisting partition

2016-05-11 Thread Spico Florin
Hello! I'm using Kafka 0.9.1. Suppose that I have created a topic "my-topic" with 1 partition.With the following code, I got StaleMetadataException in Fetcher->listOffset method and the thread is blocked in an infinite while loop (while true). I came to this error by mistake, so what to do in

Re: Zookeeper dies ... Kafka server unable to connect

2016-05-11 Thread Jaikiran Pai
On Tuesday 10 May 2016 09:29 PM, Radoslaw Gruchalski wrote: Kafka is expecting the state to be there when the zookeeper comes back. One way to protect yourself from what you see happening, is to have a zookeeper quorum. Run a cluster of 3 zookeepers, then repeat your exercise. Kafka will

Re: Kafka Connect tasks consumers issue

2016-05-11 Thread Matteo Luzzi
Hi again, I was able to reproduce the bug in the same scenario (two workers on separate machines) just by deleting the connector from the Rest API and then restarting it again. I also got this error on one of the workers : [2016-05-11 11:29:47,034] INFO 172.17.42.1 - - [11/May/2016:11:29:45 +]

Re: Kafka Connect tasks consumers issue

2016-05-11 Thread Matteo Luzzi
Hi Liquan, thanks for the fast response. I'm able to reproduce the error by having two workers running on two different machines. If I restart one of the two worker, the failover logic correctly detects the failure and shut down the tasks on the healthy worker for rebalancing. When the failed

ConsumerGroupCommand in Kafka 0.8.2

2016-05-11 Thread Joe San
In version 0.9.0.0, we have this beautiful command that would show the offset and the lag in a println as: println("%s, %s, %s, %s, %s, %s, %s" .format("GROUP", "TOPIC", "PARTITION", "CURRENT OFFSET", "LOG END OFFSET", "LAG", "OWNER")) is there an equivalent command that I could use for the

Re: Kafka Connect tasks consumers issue

2016-05-11 Thread Liquan Pei
Hi Matteo, Glad to hear that you are building a connector. To better understand the issue, can you provide the exact steps to re-produce the issue? One thing I am confused is that when one worker is shutdown, you don't need to restart the connector through the rest API, the failover logic should

Kafka Connect tasks consumers issue

2016-05-11 Thread Matteo Luzzi
Hi, I'm working on a custom implementation of a sink connector for Kafka Connect framework. I'm testing the connector for fault tolerance by killing the worker process and restarting the connector through the Rest API and occasionally I notice that some tasks don't receive anymore messages from

Re: Can we delete topic in kafka

2016-05-11 Thread Snehalata Nagaje
I am using 0.9.0.1 - Original Message - From: "Jörg Wagner" To: users@kafka.apache.org Sent: Wednesday, May 11, 2016 1:20:37 PM Subject: Re: Can we delete topic in kafka Depending on your version of kafka it may or may not work. Before 0.8.2 it didn't work

RE: Zookeeper dies ... Kafka server unable to connect

2016-05-11 Thread Paolo Patierno
I tried with persisten volume as Christian suggested and it works great for me. Thanks ! Btw I need to explore the zookeeper cluster solution as well. Paolo. Paolo PatiernoSenior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoTMicrosoft Azure Advisor Twitter :

Re: Backing up Kafka data and using it later?

2016-05-11 Thread Gerard Klijs
You could create a docker image with a kafka installation, and start a mirror maker in it, you could set the retention time for it to infinite, and mount the data volume. With the data you could always restart the docker, en mirror it to somewhere else. Not sure that's what you want, but it's an

Re: Can we delete topic in kafka

2016-05-11 Thread Jan Omar
You have to allow topic deletion in server.properties first. delete.topic.enable = true Regards Jan > On 11 May 2016, at 09:48, Snehalata Nagaje > wrote: > > > > Hi , > > Can we delete certain topic in kafka? > > I have deleted using command > >

Re: Can we delete topic in kafka

2016-05-11 Thread Jörg Wagner
Depending on your version of kafka it may or may not work. Before 0.8.2 it didn't work afaik and on 0.8.2 it works unrealiably from my experience. Can't comment on 0.9 forward. Cheers On 11.05.2016 09:48, Snehalata Nagaje wrote: Hi , Can we delete certain topic in kafka? I have deleted

Can we delete topic in kafka

2016-05-11 Thread Snehalata Nagaje
Hi , Can we delete certain topic in kafka? I have deleted using command ./kafka-topics.sh --delete --topic topic_billing --zookeeper localhost:2181 It says topic marked as deletion, but it does not actually delete topic. Thanks, Snehalata

Re: Kafka 9 version offset storage mechanism changes

2016-05-11 Thread Spico Florin
Hi! In Gwen's answer there is nothing related with the simple API consumer and the relation with ZK. But still the simple consumer API uses ZK to store offsets, isn't it? I look forward for your answer. Regards, Florin On Wed, May 11, 2016 at 2:11 AM, R Krishna wrote: >

Re: Consumer stopped after reading some messages

2016-05-11 Thread sahitya agrawal
Thanks Alex and John , it is very helpful. Regards, Sahitya Agrawal On Wed, May 11, 2016 at 4:14 AM, Alex Loddengaard wrote: > Hi Sahitya, > > I wonder if your consumers are experiencing soft failures because they're > busy processing a large collection of messages and not