kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jun Rao <jun...@gmail.com>
Subject Re: consuming only half the messages produced
Date Thu, 02 May 2013 14:56:06 GMT
In this case, you have a replication factor of 1. So, each partition has
only 1 replica.

There are two possibilities: (1) The producer didn't send all messages. (2)
The producer successfully sent all messages, but the consumer didn't
receive all messages. Could you first check the producer log and make sure
there are no errors? If so, check the consumer log. You should see sth like
"Consumer ???? selected partitions : ". See if all partitions are selected.
Also, check if there is any error in the consumer log.

To understand leader and isr, you can read the replication design in
http://kafka.apache.org/08/design.html

Thanks,

Jun


On Thu, May 2, 2013 at 6:32 AM, Rob Withers <reefedjib@gmail.com> wrote:

> Regarding the partitions created:
> >         topic: unittest-test-msg        partition: 0    leader: 0
> replicas: 0     isr: 0
> >         topic: unittest-test-msg        partition: 1    leader: 1
> replicas: 1     isr: 1
> >         topic: unittest-test-msg        partition: 2    leader: 0
> replicas: 0     isr: 0
> >         topic: unittest-test-msg        partition: 3    leader: 1
> replicas: 1     isr: 1
>
> Is this saying that partitions 1 and 3, both on the same broker, are both
> the Leader partitions AND the Replica partitions?  I do not understand the
> significance of ISR, either.   How should this be read?
>
> Keeping in mind that these were auto-created, unsure of the implications, I
> will be carefully reading the following, next:
> https://cwiki.apache.org/KAFKA/kafka-replication.html.
>
> thanks,
> rob
>
> > -----Original Message-----
> > From: Rob Withers [mailto:reefedjib@gmail.com]
> > Sent: Thursday, May 02, 2013 7:20 AM
> > To: 'users@kafka.apache.org'
> > Subject: RE: consuming only half the messages produced
> >
> > Yes, I mean we can only consume half the messages produced.  I followed
> the
> > high-level consumer example here:
> > https://cwiki.apache.org/confluence/display/KAFKA/Consumer+Group+Examp
> > le.
> >
> > Let me give a more complete scenario:
> >
> > - We run 3 zookeepers
> > - We run 2 brokers
> > - We do not have a topic defined, but we have enabled topic auto-creation
> > (with a replication factor of 2? must check this)
> > - We connect the producer to both brokers (pocmsg5:9092,pocmsg6:9092)
> > - We stuff the topic into the KeyedMessage key with no Partitioner.  I
> was
> not
> > aware of the use of the key until last night.
> > - We generate 10 messages
> > - Topic auto-creation results in the following partitions:
> >         topic: unittest-test-msg        partition: 0    leader: 0
> replicas: 0     isr: 0
> >         topic: unittest-test-msg        partition: 1    leader: 1
> replicas: 1     isr: 1
> >         topic: unittest-test-msg        partition: 2    leader: 0
> replicas: 0     isr: 0
> >         topic: unittest-test-msg        partition: 3    leader: 1
> replicas: 1     isr: 1
> > - We construct a single Kafka stream by calling createStreams with a
> > zookeeper (pocmsg5:2181) and one thread
> >         public <K,V> Map<String, List<KafkaStream<K,V>>>
> > createMessageStreams(
> >                         Map<String, Integer> topicCountMap,
> >                         Decoder<K> keyDecoder,
> >                         Decoder<V> valueDecoder)
> > - We consume only half the messages
> > - It looks as if partitions 0 and 2 are on pocmsg5, while partitions 1
> and
> 3 are
> > on pocmsg6.
> >
> > Is it best to view the situation as 2 partitions, each a leader, with a
> replica
> > follower for each?
> > which partitions are leaders and which are replicas?
> > What happened with auto-creation and production and partitioning?
> > Which partition(s) is the zookeeper pointing the high-level consumer to
> read
> > from?
> >
> > thanks,
> > rob
> >
> > > -----Original Message-----
> > > From: Jun Rao [mailto:junrao@gmail.com]
> > > Sent: Wednesday, May 01, 2013 11:15 PM
> > > To: users@kafka.apache.org
> > > Subject: Re: consuming only half the messages produced
> > >
> > > Partition is different from replicas. A topic can have one or more
> > > partitions and each partition can have one or more replicas. A
> > > consumer consumes data at partition level. In other words, a consumer
> > > gets the same data no matter how many replicas are there.
> > >
> > > When you say the consumer only gets half of the messages, do you mean
> > > that it gets half of the messages that are produced?
> > >
> > > You may want to take a look at the consumer example in
> > > http://kafka.apache.org/08/api.html
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, May 1, 2013 at 7:14 PM, Rob Withers <reefedjib@gmail.com>
> > wrote:
> > >
> > > > Running a consumer group (createStreams()), pointing to the
> > > > zookeeper and with the topic and 1 consumer thread, results in only
> > > > half the messages being consumed.  The topic was auto-created, with
> > > > a replication factor of 2, but the producer was configured to
> > > > produce to
> > > > 2 brokers and so 4 partitions resulted.  Are half getting sent to
> > > > one leader, in one broker, and the other half getting sent to
> > > > another leader, in the other broker, but the consumer stream is only
> > > > reading from one leader from the zk?  Shouldn't there only be one
> leader?
> > > >
> > > >
> > > >
> > > > thanks,
> > > >
> > > > rob
> > > >
> > > >
> > > >
> > > >
>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message