Hello Sameer,

I looked through your code, and here is what I figured: in 0.11 version we
added the exactly-once feature (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
)

Which uses the transaction log (internal topic named "__transaction_state")
that has a default replication of 3 (that will overwrite your global config
value of 2). Then at around 12:30, the leader of the transation log
partition kicked both replicas of 190 and 192 out of the replica:

[2017-09-05 12:30:31,256] INFO [GroupCoordinator 193]: Preparing to
rebalance group KafkaCache_TEST15 with old generation 14
(__consumer_offsets-27) (kafka.coordinator.group.GroupCoordinator)

*[2017-09-05 12:30:41,510] INFO Partition [__transaction_state,9] on broker
193: Shrinking ISR from 193,192,190 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,513] INFO Partition [__transaction_state,6] on broker
193: Shrinking ISR from 193,190,192 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,514] INFO Partition [__transaction_state,3] on broker
193: Shrinking ISR from 193,192,190 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,515] INFO Partition [__transaction_state,18] on
broker 193: Shrinking ISR from 193,190,192 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,515] INFO Partition [__transaction_state,15] on
broker 193: Shrinking ISR from 193,192,190 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,516] INFO Partition [__transaction_state,12] on
broker 193: Shrinking ISR from 193,190,192 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,516] INFO Partition [__consumer_offsets,12] on broker
193: Shrinking ISR from 193,192,190 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,517] INFO Partition [__consumer_offsets,15] on broker
193: Shrinking ISR from 193,192,190 to 193 (kafka.cluster.Partition)*

*[2017-09-05 12:30:41,517] INFO Partition [__transaction_state,24] on
broker 193: Shrinking ISR from 193,190,192 to 193 (kafka.cluster.Partition)*

At the mean time, both replicas of 190 and 192 seems to be timed out on
their fetch requests (note the big timestamp gap in the logs):

[2017-09-05 12:26:21,130] INFO Rolled new log segment for 'AdServe-4' in 1
ms. (kafka.log.Log)
[2017-09-05 12:30:59,046] WARN [ReplicaFetcherThread-2-193]: Error in fetch
to broker 193, request (type=FetchRequest, replicaId=190, maxWait=500,
minBytes=1, maxBytes=10485760, fetchData={__consumer_offsets-21=(offset=0,
logStartOffset=0, maxBytes=1048576)

...

[2017-09-05 12:28:37,514] INFO Deleting index
/data1/kafka/AdServe-5/00000000000405000294.timeindex.deleted
(kafka.log.TimeIndex)
[2017-09-05 12:30:59,042] WARN [ReplicaFetcherThread-2-193]: Error in fetch
to broker 193, request (type=FetchRequest, replicaId=192, maxWait=500,
minBytes=1, maxBytes=10485760, fetchData={__consumer_offsets-21=(offset=0,
logStartOffset=0, maxBytes=1048576)



This caused the NotEnoughReplicasException since any appends to the
transaction logs are required "acks=all, and min.isr=num.replicas".

*[2017-09-05 12:32:11,612] ERROR [Replica Manager on Broker 193]: Error
processing append operation on partition __transaction_state-18
(kafka.server.ReplicaManager)*

*org.apache.kafka.common.errors.NotEnoughReplicasException: Number of
insync replicas for partition __transaction_state-18 is [1], below required
minimum [3]*

Upon seeing this error, the transaction coordinator should retry appending,
but if the retry never succeeds it will be blocked. I did not see the
Streams API client-side logs and so cannot tell for sure, why this caused
the Streams app to fail as well. A quick question: did you enable
`processing.mode=exactly-once` on your streams app?


Guozhang




On Fri, Sep 8, 2017 at 1:34 AM, Sameer Kumar <sam.kum.w...@gmail.com> wrote:

> Hi All,
>
>
> Any thoughts on the below mail.
>
> -Sameer.
>
> On Wed, Sep 6, 2017 at 12:28 PM, Sameer Kumar <sam.kum.w...@gmail.com>
> wrote:
>
> > Hi All,
> >
> > I want to report a scenario wherein my running 2 different instances of
> my
> > stream application caused my brokers to crash and eventually my stream
> > application as well. This scenario only happens when my brokers run on
> > Kafka11, everything works fine if my brokers are on Kafka 10..2 and
> stream
> > application on Kafka11.
> >
> > I am attaching herewith the logs in a zipped format.
> >
> > The cluster configuration
> > 3 nodes(190,192,193) , Kafka 11
> > Topic Replication Factor - 2
> >
> > App configuration
> > Kafka 11 streams.
> >
> >
> > The error I saw on 193 server was org.apache.kafka.common.errors.
> NotEnoughReplicasException:
> > Number of insync replicas for partition __transaction_state-18 is [1],
> > below required minimum [3]. Both 192,190 servers reported errors on
> failure
> > to read information from 193.
> >
> > Please look for the time around 12:30-12:32 to find the relevant logs.
> Let
> > me know if you need some other information.
> >
> >
> > Regards,
> > -Sameer.
> >
>



-- 
-- Guozhang

Reply via email to