Kafka slow consumer

2016-08-29 Thread Oleg Zhurakousky
Hi all Have a question about the scenario where consumer that is consuming Kafka records is not very fast (regardless of the reason). And yes I know about certain configuration properties on both server and consumer which help with mitigating the effects, so I just simply want to confirm that

Re: consumer.poll() hangs indefinitely in docker container

2016-08-16 Thread Oleg Zhurakousky
://issues.apache.org/jira/browse/KAFKA-3539 Cheers Oleg On Aug 16, 2016, at 5:06 AM, Jaikiran Pai <jai.forums2...@gmail.com<mailto:jai.forums2...@gmail.com>> wrote: On Friday 12 August 2016 08:45 PM, Oleg Zhurakousky wrote: It hangs indefinitely in any container. I don't think that's accurate.

Re: consumer.poll() hangs indefinitely in docker container

2016-08-12 Thread Oleg Zhurakousky
It hangs indefinitely in any container. It’s a known issue and has been brought up many times on this list, yet there is not fix for it. The problem is with the fact that while poll() attempts to create an elusion that it is async and even allows you to set a timeout it is essentially very

Re: KafkaConsumer blocks indefinitely when server settings are wrong

2016-08-06 Thread Oleg Zhurakousky
ack-Postava <e...@confluent.io> wrote: > > This is unfortunate, but a known issue. See > https://issues.apache.org/jira/browse/KAFKA-1894 The producer suffers from > a similar issue with its initial metadata fetch on the first send(). > > -Ewen > > On Thu, Jul 28, 2

Re: Kafka Consumer poll

2016-08-02 Thread Oleg Zhurakousky
Also keep in mind that unfortunately KafkaConsumer.poll(..) will deadlock regardless of the timeout if connection to the broker can not be established and won't react to thread interrupts. This essentially means that the only way to exit is to kill jvm. This is all because Kafka fetches topic

Re: KafkaConsumer blocks indefinitely when server settings are wrong

2016-07-28 Thread Oleg Zhurakousky
no joy. Cheers Oleg On Jul 28, 2016, at 3:43 PM, Oleg Zhurakousky <ozhurakou...@hortonworks.com<mailto:ozhurakou...@hortonworks.com>> wrote: So I have KafkaConsumer that is deliberately set with server properties pointing to non-running broker. Doing KafkaConsumer.poll(100) block

KafkaConsumer blocks indefinitely when server settings are wrong

2016-07-28 Thread Oleg Zhurakousky
So I have KafkaConsumer that is deliberately set with server properties pointing to non-running broker. Doing KafkaConsumer.poll(100) blocks infinitely even though ‘fetch.max.wait.ms’ is set to 1 millisecond. Basically I am trying to fail when connection is not possible. Any idea how to

0.10 Kafka with 0.9 brokers

2016-05-24 Thread Oleg Zhurakousky
With 0.10 Kafka being announced, are there any known compatibility issues with 0.9 brokers? Thanks Oleg

Re: Using JMS and Multi-Protocol support - ActiveMQ to Kafka

2016-05-09 Thread Oleg Zhurakousky
I think you are confusing wire-based vs. API-based messaging. ActiveMQ is API-based messaging where API is Java Messaging Service (JMS) hence restrictions to both sides (producer/consumer) + broker being Java-based (although it doesn't have to be in theory). Kafka, AMQP and others are

Re: KafkaProducer block on send

2016-05-04 Thread Oleg Zhurakousky
> >> I think changes of this sort (design changes as opposed to bugs) typically >> go through a KIP process before work is assigned. You might consider >> starting a KIP discussion and see if there is interest in pursuing your >> proposed changes. >> >> -Dana &

Re: KafkaProducer block on send

2016-05-04 Thread Oleg Zhurakousky
Indeed it is. Oleg > On May 4, 2016, at 10:54 AM, Paolo Patierno wrote: > > It's sad that after almost one month it's still "unassigned" :-( > > Paolo PatiernoSenior Software Engineer (IoT) @ Red Hat > Microsoft MVP on Windows Embedded & IoTMicrosoft Azure Advisor >

Re: KafkaProducer block on send

2016-05-04 Thread Oleg Zhurakousky
Sure Here are both: https://issues.apache.org/jira/browse/KAFKA-3539 https://issues.apache.org/jira/browse/KAFKA-3540 On May 4, 2016, at 3:24 AM, Paolo Patierno > wrote: Hi Oleg, can you share the JIRA link here because I totally agree with you.

Re: KafkaProducer stuck in send() call

2016-04-21 Thread Oleg Zhurakousky
Thank you Vinay Will give it a shot and see if it happens again. Oleg > On Apr 21, 2016, at 12:26 PM, vinay sharma wrote: > > Hi Oleg, > > There are a few configurations of kafka producer that are worth considering > in your case.:- ProducerConfig.RETRIES_CONFIG >

KafkaProducer stuck in send() call

2016-04-21 Thread Oleg Zhurakousky
So we have the following scenario Kafka broker (Kerberized) goes into the state of unresponsiveness while KafkaProducer issued a send() call. Regardless of the reasons why the broker ends up in this state my question is really about KafkaProducer. It continues to retry (see stack trace below)

Re: Control the amount of messages batched up by KafkaConsumer.poll()

2016-04-12 Thread Oleg Zhurakousky
"max.poll.records" is getting introduced in > upcoming 0.10 release. > This property can be used to control the no. of records in each poll. > > > Manikumar > > On Tue, Apr 12, 2016 at 6:26 PM, Oleg Zhurakousky < > ozhurakou...@hortonworks.com> wrote:

Control the amount of messages batched up by KafkaConsumer.poll()

2016-04-12 Thread Oleg Zhurakousky
Is there a way to specify in KafkaConsumer up to how many messages do I want o receive? I am operation under premise that Consumer.poll(..) returns a batch of messages, but not sure if there is a way to control batch amount. Cheers Oleg

Re: "Close the consumer, waiting indefinitely for any needed cleanup."

2016-04-11 Thread Oleg Zhurakousky
required. > > -Dana > > On Mon, Apr 11, 2016 at 1:17 PM, Oleg Zhurakousky > <ozhurakou...@hortonworks.com> wrote: >> Dana >> Everything your are saying does not answer my question of how to interrupt a >> potential deadlock artificially forced upon users of

Re: "Close the consumer, waiting indefinitely for any needed cleanup."

2016-04-11 Thread Oleg Zhurakousky
pinions. > > -Dana > > On Mon, Apr 11, 2016 at 12:45 PM, Oleg Zhurakousky > <ozhurakou...@hortonworks.com> wrote: >> The subject line is from the javadoc of the new KafkaConsumer. >> Is this for real? I mean I am hoping the use of ‘indefinitely' is a typo. >

"Close the consumer, waiting indefinitely for any needed cleanup."

2016-04-11 Thread Oleg Zhurakousky
The subject line is from the javadoc of the new KafkaConsumer. Is this for real? I mean I am hoping the use of ‘indefinitely' is a typo. In any event if it is indeed true, how does one break out of indefinitely blocking consumer.close() invocation? Cheers Oleg

Re: KafkaProducer block on send

2016-04-11 Thread Oleg Zhurakousky
Dana Thanks for the explanation, but it sounds more like a workaround since everything you describe could be encapsulated within the Future itself. After all it "represents the result of an asynchronous computation" executor.submit(new Callable() { @Override public RecordMetadata

KafkaProducer block on send

2016-04-07 Thread Oleg Zhurakousky
I know it’s been discussed before, but that conversation never really concluded with any reasonable explanation, so I am bringing it up again as I believe this is a bug that would need to be fixed in some future release. Can someone please explain the rational for the following code in

Re: Partition size for topic

2016-04-02 Thread Oleg Zhurakousky
-by-one error is the issue, where you're specifying > `numPartitions` instead of a value in [0, `numPartitions`-1]? > > -Ewen > > On Mon, Mar 28, 2016 at 2:20 PM, Oleg Zhurakousky < > ozhurakou...@hortonworks.com> wrote: > >> Hi >> >> It seems there ar

Deadlock when committing offsets

2016-04-01 Thread Oleg Zhurakousky
Hi Was wondering what could be the causes of what appears to be an intermittent deadlock when calling Consumer.commitOffsets(..) Below are the few relevant thread dump segments: "StandardProcessScheduler Thread-7" Id=115 BLOCKED on java.lang.Object@2baae51 at

Re: Using the new consumer client API 0.0.9

2016-03-30 Thread Oleg Zhurakousky
Jason Are those API changes you mentioned binary compatible with previous release? Cheers Oleg > On Mar 30, 2016, at 12:03 PM, Jason Gustafson wrote: > > Hi Prabhakar, > > We fixed a couple critical bugs in the 0.9.0.1 release, so you should > definitely make sure to use

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

2016-03-30 Thread Oleg Zhurakousky
ailable, we put msg into a queue and drain the queue from a diff >> thread. >> >> Thanks, >> Steven >> >> >> On Tue, Mar 29, 2016 at 4:59 AM, Oleg Zhurakousky < >> ozhurakou...@hortonworks.com> wrote: >> >>> I agree and cons

Re: Consumer thread is waiting forever, not returning any objects

2016-03-29 Thread Oleg Zhurakousky
; > > processor.processFile(record.value()); > > > System.out.println("&&&&&&"+record.value()); > > } > > } > > > } catch (Throwable e) { > > e.printStackTrace(); > > System.out.println("eror in poll

Re: Consumer thread is waiting forever, not returning any objects

2016-03-29 Thread Oleg Zhurakousky
Ratha It appears you have couple of issues here, so I’ll start with the consumer first. If you do a search on this mailing list on “Consumer deadlock” in the subject you’ll find a thread where similar symptoms were discussed. Basically the hasNext() method you mentioned is implemented as a

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

2016-03-29 Thread Oleg Zhurakousky
I agree and considering that send(..) method returns Future one would argue it must never block, otherwise what’s the point of returning Future if you remove user’s ability to control how long are they willing to wait and what to do when certain types of exception arise. Nevertheless it does

Partition size for topic

2016-03-28 Thread Oleg Zhurakousky
Hi It seems there are several ways to get to the same number in Kafka API. In Kafka Partitioner which is invoked by KafkaProducer we have this public int partition(ProducerRecord record, Cluster cluster) { List partitions = cluster.partitionsForTopic(record.topic());

Stopping background ZK reconnect thread

2016-03-28 Thread Oleg Zhurakousky
Guys When stopping ‘ConsumerConnector’ by calling shutdown() one would assume that it is stopped, but it turns out it is not, since there is a background thread running infinitely that attempts to reconnect to ZK. The best way to reproduce it is to stop ZK and you will immediately start seeing

Re: Consumer deadlock

2016-03-03 Thread Oleg Zhurakousky
hod (though it currently has some limitations > to how it enforces that timeout). > > -Ewen > > On Wed, Mar 2, 2016 at 11:24 AM, Oleg Zhurakousky < > ozhurakou...@hortonworks.com> wrote: > >> Guys >> >> We have a consumer deadlock and here is

Connecting to secure Kafka

2016-02-29 Thread Oleg Zhurakousky
So, my Kafka is kerberized and all works well However when trying to connect with missing SASL properties Kafka doesn’t fail. It simply hangs indefinitely. Here is the example code; Properties props = new Properties(); props.put("bootstrap.servers",

Re: Kerberized Kafka setup issues

2016-02-24 Thread Oleg Zhurakousky
changed it all went fine! Cheers Oleg > On Feb 23, 2016, at 6:09 PM, Oleg Zhurakousky <ozhurakou...@hortonworks.com> > wrote: > > Well, I am running on the same machine, so I say yes > > Sent from my iPhone > >> On Feb 23, 2016, at 18:05, Martin Gainty <mgai...

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
’ll poke around what >> the differences are and if find the issue will post. >> Thanks for your help anyway. >> >> Cheers >> Oleg >> On Feb 23, 2016, at 4:06 PM, Oleg Zhurakousky >> <ozhurakou...@hortonworks.com<mailto:ozhurakou...@hortonworks.com>

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
On Feb 23, 2016, at 4:06 PM, Oleg Zhurakousky <ozhurakou...@hortonworks.com<mailto:ozhurakou...@hortonworks.com>> wrote: Yeah, I noticed the localhost as well, but I’ve changed it since to FQDN and it is still the same including 'sname is zookeeper/localh...@oleg.com<mailto:z

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
s like. Did you > use the hostname or localhost > -Harsha > > On Tue, Feb 23, 2016, at 12:01 PM, Oleg Zhurakousky wrote: >> Still digging, but here is more info that may help >> >> 2016-02-23 14:59:24,240] INFO zookeeper state changed (SyncConnected) >> (org.

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
.@oleg.com sname is zookeeper/localh...@oleg.com msgType is 30 > On Feb 23, 2016, at 2:46 PM, Oleg Zhurakousky <ozhurakou...@hortonworks.com> > wrote: > > No joy. the same error > > KafkaServer { >com.sun.security.auth.module.Krb5LoginMod

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
keyTab="/home/oleg/kafka_2.10-0.9.0.1/config/security/kafka.keytab" >>principal="kafka/ubuntu.oleg@oleg.com"; >> }; >> >> On Tue, Feb 23, 2016, at 11:24 AM, Oleg Zhurakousky wrote: >>> More info >>> >>>

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
More info I am starting both services as myself ‘oleg’. Validated that both key tab files are readable. o I am assuming Zookeeper is started as ‘zookeeper’ and Kafka as ‘kafka’ Oleg > On Feb 23, 2016, at 2:22 PM, Oleg Zhurakousky <ozhurakou...@hortonworks.com> > wrote:

Re: Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
s important that serviceName >must match the principal name with which zookeeper is running. >Whats the principal name zookeeper service is running with. > -Harsha > > On Tue, Feb 23, 2016, at 11:01 AM, Oleg Zhurakousky wrote: >> Hey guys, first post here so

Kerberized Kafka setup issues

2016-02-23 Thread Oleg Zhurakousky
Hey guys, first post here so bare with me Trying to setup Kerberized Kafka 0.9.0.. Followed the instructions here http://kafka.apache.org/documentation.html#security_sasl and i seem to be very close, but not quite there yet. ZOOKEEPER Starting Zookeeper seems to be OK (below is the relevant