Yes, mapping an iterator interface to a networked service can have some
drawbacks :) Providing a simple, familiar interface to Kafka is challenging
-- if you didn't block and hasNext() returned false, what would you do
after that? That return value doesn't indicate that the iterable for a
Kafka topic ended, just that there wasn't anything available *immediately*.
But once hasNext() returns false, it's expected that it will always return
false and that there won't be more elements from the Iterator. With a Kafka
topic, there isn't really an end to the collection, there just might not be
any data available immediately.

When it comes down to it, there's a mismatch between the Java Iterator
interface and what Kafka is trying to provide. That's one of the reasons
the consumer interface has been rethought and looks significantly different
in the new consumer:
http://kafka.apache.org/090/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html

-Ewen

On Thu, Mar 3, 2016 at 1:51 AM, Oleg Zhurakousky <
ozhurakou...@hortonworks.com> wrote:

> Thank you Ewan, I'll give it a shot, but I am still puzzled as to the
> reasoning behind implementing hasNext() in a manner that it can block.
> Isn't that counter to the whole point of the method? You either have it or
> you not. Blocking in the off chance one may have it in some time in the
> future simply means it's not there at the moment, so why not return false
> and let user retry?
>
> Sent from my iPhone
>
> > On Mar 3, 2016, at 03:51, Ewen Cheslack-Postava <e...@confluent.io>
> wrote:
> >
> > Take a look at the consumer.timeout.ms setting if you don't want the
> > iterator to block indefinitely.
> >
> > And a better long term solution is to switch to the new consumer, but
> that
> > obviously requires much more significant code changes. The new consumer
> API
> > is a single-threaded poll-based API where you can always specify timeouts
> > to the consumer's poll() method (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 the relevant dump:
> >>
> >> "Timer-Driven Process Thread-10" Id=76 TIMED_WAITING  on
> >>
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@39775787
> >>        at sun.misc.Unsafe.park(Native Method)
> >>        at
> >> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> >>        at
> >>
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> >>        at
> >>
> java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
> >>        at
> >> kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:65)
> >>        at
> >> kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33)
> >>        at
> >> kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:66)
> >>        at
> kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:58)
> >>        . . . . .
> >>
> >> What worries me is the fact that ‘hasNext’ is essentially a blocking
> >> operation. I can’t seem to find a single reason when it would be useful,
> >> hence I am calling it a bug, but hopefully someone can clarify.
> >> Kafka version is 0.8.*
> >>
> >> Cheers
> >> Oleg
> >
> >
> > --
> > Thanks,
> > Ewen
>



-- 
Thanks,
Ewen

Reply via email to