[ 
https://issues.apache.org/jira/browse/KAFKA-6003?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16192028#comment-16192028
 ] 

ASF GitHub Bot commented on KAFKA-6003:
---------------------------------------

GitHub user apurvam opened a pull request:

    https://github.com/apache/kafka/pull/4020

    KAFKA-6003: Accept appends on replicas and when rebuilding the log 
unconditionally

    This is a port of #4004 for the 0.11.0 branch.
    
    With this patch so that we _only_ validate appends which originate
    from the client. In general, once the append is validated and written to
    the leader the first time, revalidating it is undesirable since we can't
    do anything if validation fails, and also because it is hard to maintain
    the correct assumptions during validation, leading to spurious
    validation failures.
    
    For example, when we have compacted topics, it is possible for batches
    to be compacted on the follower but not on the leader. This case would
    also lead to an OutOfOrderSequencException during replication. The same
    applies to when we rebuild state from compacted topics: we would get
    gaps in the sequence numbers, causing the OutOfOrderSequence.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/apurvam/kafka 
KAKFA-6003-0.11.0-handle-unknown-producer-on-replica

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/4020.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4020
    
----
commit 0a6a0213c091c8e6b6a9c5ce7655b7e0d06c9db0
Author: Apurva Mehta <apu...@confluent.io>
Date:   2017-10-04T20:42:17Z

    KAFKA-6003: Accept appends on replicas and when rebuilding state from
    the log unconditionally.
    
    With this patch so that we _only_ validate appends which originate
    from the client. In general, once the append is validated and written to
    the leader the first time, revalidating it is undesirable since we can't
    do anything if validation fails, and also because it is hard to maintain
    the correct assumptions during validation, leading to spurious
    validation failures.
    
    For example, when we have compacted topics, it is possible for batches
    to be compacted on the follower but not on the leader. This case would
    also lead to an OutOfOrderSequencException during replication. The same
    applies to when we rebuild state from compacted topics: we would get
    gaps in the sequence numbers, causing the OutOfOrderSequence.

----


> Replication Fetcher thread for a partition with no data fails to start
> ----------------------------------------------------------------------
>
>                 Key: KAFKA-6003
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6003
>             Project: Kafka
>          Issue Type: Bug
>          Components: replication
>    Affects Versions: 0.11.0.1
>            Reporter: Stanislav Chizhov
>            Assignee: Apurva Mehta
>            Priority: Blocker
>             Fix For: 1.0.0, 0.11.0.2
>
>
> If a partition of a topic with idempotent producer has no data on 1 of the 
> brokers, but it does exist on others and some of the segments for this 
> partition have been already deleted replication thread responsible for this 
> partition on the broker which has no data for it fails to start with out of 
> order sequence exception:
> {code}
> [2017-10-02 09:44:23,825] ERROR [ReplicaFetcherThread-2-4]: Error due to 
> (kafka.server.ReplicaFetcherThread)
> kafka.common.KafkaException: error processing data for partition 
> [stage.data.adevents.v2,20] offset 1660336429
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:203)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:174)
>         at scala.Option.foreach(Option.scala:257)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:174)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:171)
>         at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>         at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply$mcV$sp(AbstractFetcherThread.scala:171)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:171)
>         at 
> kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:171)
>         at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:213)
>         at 
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:169)
>         at 
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64)
> Caused by: org.apache.kafka.common.errors.OutOfOrderSequenceException: 
> Invalid sequence number for new epoch: 0 (request epoch), 154277489 (seq. 
> number)
> {code}
> We run kafka 0.11.0.1 and we ran into the situation when 1 of replication 
> threads was stopped for few days, while everything else on that broker was 
> functional. This is our staging cluster and retention is less than a day, so 
> everything for partitions for which replication thread was down was cleaned 
> up. At the moment we have a broker which cannot start replication for few 
> partitions. I was also able to reproduce in my local test environment.
> Another possible use case when this might cause real pain is disk failure or 
> any situation when previously deleting all the data for the partition on a 
> broker helped - since it would just fetch all the data from other replicas. 
> Now it does not work for topics with idempotent producers. It might also 
> affect other not-idempotent topics if those are unlucky to share same 
> replication fetcher thread. 
> This seems to be caused by this logic: 
> https://github.com/apache/kafka/blob/0.11.0.1/core/src/main/scala/kafka/log/ProducerStateManager.scala#L119
> and might be fixed in the scope of 
> https://issues.apache.org/jira/browse/KAFKA-5793.
> However any hints on how to get those partition to fully replicated state are 
> highly appreciated.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to