----------------------------------------------------------- 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. Changes ------- Addressed Neha's and June's comments. Main changes: * appendMessagesToLeader also checks minISR and acks to avoid writing to log when there are not enough replicas. This means that appendMessagesToLeader now takes an extra argument with acks. I defaulted to acks=0 to retain previous behavior in cases when this is not actually a producer request (Part of the compaction code also appends messages). * We now validate that acks are in (-1,0,1). For the new producer I added an extra validator because acks was a string and we can't change that without breaking clients. The string validator will be useful when we switch to enum. * However, it looks like the new producer does not use the validator, except on the default value. This is a general problem, so I didn't fix it here, but the new producer still accepts acks>1 * If we catch minISR issue before appending message, we throw a NotEnoughReplica exception and there are no duplicates. If we catch minISR issue after appending to log (while waiting for acks), we throw NotEnoughReplicaAfterAppend exception, so the client will be aware of possible duplicates. The new exception should be rare, and I could not figure out a way to test it (unit or other), so its also untested. Repository: kafka Description ------- KAFKA-1555: provide strong consistency with reasonable availability Diffs (updated) ----- 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