> On Oct. 2, 2014, 4:08 a.m., Jun Rao wrote:
> > core/src/main/scala/kafka/cluster/Partition.scala, lines 372-373
> > <https://reviews.apache.org/r/25886/diff/7/?file=709987#file709987line372>
> >
> >     Hmm, this code looks a bit weird. It's weird in that a topic level 
> > config only takes effect under certain producer side config. My feeling is 
> > that if we make min.isr a topic level config, then it should work in the 
> > same way independent of the producer side config. So, perhaps it's better 
> > to just check inSyncSize < minIsr, irrespective of the ack. What do you 
> > think?
> 
> Gwen Shapira wrote:
>     Agree its weird. However, that was pretty much what we agreed on in the 
> Jira discussion - if a producer specifies acks=0 and acks=1, they don't care 
> about reliability that much, they certainly don't care about size of ISR 
> (since they don't expect any acks from ISR). I'm pretty sure we decided that 
> min.isr only makes sense with acks=-1, so thats what I implemented.
>     
>     Since it does look out of place here, and I'm a bit worried about 
> non-producer calls to this function, I'll try to move this check higher up 
> the call stack.
>     
>     Does that make sense?
> 
> Gwen Shapira wrote:
>     Actually, we can't move it up the call stack since we don't have access 
> to the min.isr value farther up (conf only exists on the leader replica for a 
> partition). I can't think of a better way to fulfill the requirement of 
> min.isr only applies when acks=-1, which IMO is an important requirement.

Ok. Let's leave this as it is then. Could you address the rest of the comments?


- Jun


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/25886/#review55187
-----------------------------------------------------------


On Oct. 1, 2014, 1:19 a.m., Gwen Shapira wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/25886/
> -----------------------------------------------------------
> 
> (Updated Oct. 1, 2014, 1:19 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-1555: provide strong consistency with reasonable availability
> 
> 
> Diffs
> -----
> 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> f9de4af 
>   clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java addc906 
>   
> clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java d434f42 
>   core/src/main/scala/kafka/cluster/Partition.scala ff106b4 
>   core/src/main/scala/kafka/common/ErrorMapping.scala 3fae791 
>   
> core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala 
> PRE-CREATION 
>   core/src/main/scala/kafka/common/NotEnoughReplicasException.scala 
> PRE-CREATION 
>   core/src/main/scala/kafka/log/LogConfig.scala 5746ad4 
>   core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c 
>   core/src/main/scala/kafka/server/KafkaApis.scala c584b55 
>   core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala 
> 39f777b 
>   core/src/test/scala/unit/kafka/producer/ProducerTest.scala dd71d81 
>   core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala 24deea0 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 2dbdd3c 
> 
> Diff: https://reviews.apache.org/r/25886/diff/
> 
> 
> Testing
> -------
> 
> With 3 broker cluster, created 3 topics each with 1 partition and 3 replicas, 
> with 1,3 and 4 min.insync.replicas.
> * min.insync.replicas=1 behaved normally (all writes succeeded as long as a 
> broker was up)
> * min.insync.replicas=3 returned NotEnoughReplicas when required.acks=-1 and 
> one broker was down
> * min.insync.replicas=4 returned NotEnoughReplicas when required.acks=-1
> 
> See notes about retry behavior in the JIRA.
> 
> 
> Thanks,
> 
> Gwen Shapira
> 
>

Reply via email to