Do you have unclean leader election turned on? If killing 100 is the only
way to reproduce the problem, it is possible with unclean leader election
turned on that leadership was transferred to out of ISR follower which may
not have the latest high watermark
On Sat, Oct 7, 2017 at 3:51 AM Dmitriy Vsekhvalnov <dvsekhval...@gmail.com>
wrote:

> About to verify hypothesis on monday, but looks like that in latest tests.
> Need to double check.
>
> On Fri, Oct 6, 2017 at 11:25 PM, Stas Chizhov <schiz...@gmail.com> wrote:
>
> > So no matter in what sequence you shutdown brokers it is only 1 that
> causes
> > the major problem? That would indeed be a bit weird. have you checked
> > offsets of your consumer - right after offsets jump back - does it start
> > from the topic start or does it go back to some random position? Have you
> > checked if all offsets are actually being committed by consumers?
> >
> > fre 6 okt. 2017 kl. 20:59 skrev Dmitriy Vsekhvalnov <
> > dvsekhval...@gmail.com
> > >:
> >
> > > Yeah, probably we can dig around.
> > >
> > > One more observation, the most lag/re-consumption trouble happening
> when
> > we
> > > kill broker with lowest id (e.g. 100 from [100,101,102]).
> > > When crashing other brokers - there is nothing special happening, lag
> > > growing little bit but nothing crazy (e.g. thousands, not millions).
> > >
> > > Is it sounds suspicious?
> > >
> > > On Fri, Oct 6, 2017 at 9:23 PM, Stas Chizhov <schiz...@gmail.com>
> wrote:
> > >
> > > > Ted: when choosing earliest/latest you are saying: if it happens that
> > > there
> > > > is no "valid" offset committed for a consumer (for whatever reason:
> > > > bug/misconfiguration/no luck) it will be ok to start from the
> beginning
> > > or
> > > > end of the topic. So if you are not ok with that you should choose
> > none.
> > > >
> > > > Dmitriy: Ok. Then it is spring-kafka that maintains this offset per
> > > > partition state for you. it might also has that problem of leaving
> > stale
> > > > offsets lying around, After quickly looking through
> > > > https://github.com/spring-projects/spring-kafka/blob/
> > > > 1945f29d5518e3c4a9950ba82135420dfb61e808/spring-kafka/src/
> > > > main/java/org/springframework/kafka/listener/
> > > > KafkaMessageListenerContainer.java
> > > > it looks possible since offsets map is not cleared upon partition
> > > > revocation, but that is just a hypothesis. I have no experience with
> > > > spring-kafka. However since you say you consumers were always active
> I
> > > find
> > > > this theory worth investigating.
> > > >
> > > >
> > > > 2017-10-06 18:20 GMT+02:00 Vincent Dautremont <
> > > > vincent.dautrem...@olamobile.com.invalid>:
> > > >
> > > > > is there a way to read messages on a topic partition from a
> specific
> > > node
> > > > > we that we choose (and not by the topic partition leader) ?
> > > > > I would like to read myself that each of the __consumer_offsets
> > > partition
> > > > > replicas have the same consumer group offset written in it in it.
> > > > >
> > > > > On Fri, Oct 6, 2017 at 6:08 PM, Dmitriy Vsekhvalnov <
> > > > > dvsekhval...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Stas:
> > > > > >
> > > > > > we rely on spring-kafka, it  commits offsets "manually" for us
> > after
> > > > > event
> > > > > > handler completed. So it's kind of automatic once there is
> constant
> > > > > stream
> > > > > > of events (no idle time, which is true for us). Though it's not
> > what
> > > > pure
> > > > > > kafka-client calls "automatic" (flush commits at fixed
> intervals).
> > > > > >
> > > > > > On Fri, Oct 6, 2017 at 7:04 PM, Stas Chizhov <schiz...@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > You don't have autocmmit enables that means you commit offsets
> > > > > yourself -
> > > > > > > correct? If you store them per partition somewhere and fail to
> > > clean
> > > > it
> > > > > > up
> > > > > > > upon rebalance next time the consumer gets this partition
> > assigned
> > > > > during
> > > > > > > next rebalance it can commit old stale offset- can this be the
> > > case?
> > > > > > >
> > > > > > >
> > > > > > > fre 6 okt. 2017 kl. 17:59 skrev Dmitriy Vsekhvalnov <
> > > > > > > dvsekhval...@gmail.com
> > > > > > > >:
> > > > > > >
> > > > > > > > Reprocessing same events again - is fine for us (idempotent).
> > > While
> > > > > > > loosing
> > > > > > > > data is more critical.
> > > > > > > >
> > > > > > > > What are reasons of such behaviour? Consumers are never idle,
> > > > always
> > > > > > > > commiting, probably something wrong with broker setup then?
> > > > > > > >
> > > > > > > > On Fri, Oct 6, 2017 at 6:58 PM, Ted Yu <yuzhih...@gmail.com>
> > > > wrote:
> > > > > > > >
> > > > > > > > > Stas:
> > > > > > > > > bq.  using anything but none is not really an option
> > > > > > > > >
> > > > > > > > > If you have time, can you explain a bit more ?
> > > > > > > > >
> > > > > > > > > Thanks
> > > > > > > > >
> > > > > > > > > On Fri, Oct 6, 2017 at 8:55 AM, Stas Chizhov <
> > > schiz...@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > If you set auto.offset.reset to none next time it happens
> > you
> > > > > will
> > > > > > be
> > > > > > > > in
> > > > > > > > > > much better position to find out what happens. Also in
> > > general
> > > > > with
> > > > > > > > > current
> > > > > > > > > > semantics of offset reset policy IMO using anything but
> > none
> > > is
> > > > > not
> > > > > > > > > really
> > > > > > > > > > an option unless it is ok for consumer to loose some data
> > > > > (latest)
> > > > > > or
> > > > > > > > > > reprocess it second time (earliest).
> > > > > > > > > >
> > > > > > > > > > fre 6 okt. 2017 kl. 17:44 skrev Ted Yu <
> > yuzhih...@gmail.com
> > > >:
> > > > > > > > > >
> > > > > > > > > > > Should Kafka log warning if log.retention.hours is
> lower
> > > than
> > > > > > > number
> > > > > > > > of
> > > > > > > > > > > hours specified by offsets.retention.minutes ?
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Oct 6, 2017 at 8:35 AM, Manikumar <
> > > > > > > manikumar.re...@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > normally, log.retention.hours (168hrs)  should be
> > higher
> > > > than
> > > > > > > > > > > > offsets.retention.minutes (336 hrs)?
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Oct 6, 2017 at 8:58 PM, Dmitriy Vsekhvalnov <
> > > > > > > > > > > > dvsekhval...@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Ted,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Broker: v0.11.0.0
> > > > > > > > > > > > >
> > > > > > > > > > > > > Consumer:
> > > > > > > > > > > > > kafka-clients v0.11.0.0
> > > > > > > > > > > > > auto.offset.reset = earliest
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Oct 6, 2017 at 6:24 PM, Ted Yu <
> > > > > yuzhih...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > What's the value for auto.offset.reset  ?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Which release are you using ?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Cheers
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Oct 6, 2017 at 7:52 AM, Dmitriy
> > Vsekhvalnov <
> > > > > > > > > > > > > > dvsekhval...@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > we several time faced situation where
> > > consumer-group
> > > > > > > started
> > > > > > > > to
> > > > > > > > > > > > > > re-consume
> > > > > > > > > > > > > > > old events from beginning. Here is scenario:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1. x3 broker kafka cluster on top of x3 node
> > > > zookeeper
> > > > > > > > > > > > > > > 2. RF=3 for all topics
> > > > > > > > > > > > > > > 3. log.retention.hours=168 and
> > > > > > > > offsets.retention.minutes=20160
> > > > > > > > > > > > > > > 4. running sustainable load (pushing events)
> > > > > > > > > > > > > > > 5. doing disaster testing by randomly shutting
> > > down 1
> > > > > of
> > > > > > 3
> > > > > > > > > broker
> > > > > > > > > > > > nodes
> > > > > > > > > > > > > > > (then provision new broker back)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Several times after bouncing broker we faced
> > > > situation
> > > > > > > where
> > > > > > > > > > > consumer
> > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > started to re-consume old events.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > consumer group:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1. enable.auto.commit = false
> > > > > > > > > > > > > > > 2. tried graceful group shutdown, kill -9 and
> > > > > terminating
> > > > > > > AWS
> > > > > > > > > > nodes
> > > > > > > > > > > > > > > 3. never experienced re-consumption for given
> > > cases.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > What can cause that old events re-consumption?
> Is
> > > it
> > > > > > > related
> > > > > > > > to
> > > > > > > > > > > > > bouncing
> > > > > > > > > > > > > > > one of brokers? What to search in a logs? Any
> > > broker
> > > > > > > settings
> > > > > > > > > to
> > > > > > > > > > > try?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks in advance.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > > --
> > > > > The information transmitted is intended only for the person or
> entity
> > > to
> > > > > which it is addressed and may contain confidential and/or
> privileged
> > > > > material. Any review, retransmission, dissemination or other use
> of,
> > or
> > > > > taking of any action in reliance upon, this information by persons
> or
> > > > > entities other than the intended recipient is prohibited. If you
> > > received
> > > > > this in error, please contact the sender and delete the material
> from
> > > any
> > > > > computer.
> > > > >
> > > >
> > >
> >
>

Reply via email to