showuon commented on a change in pull request #11788: URL: https://github.com/apache/kafka/pull/11788#discussion_r815260093
########## File path: clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java ########## @@ -461,27 +467,53 @@ private void postProcessAndValidateIdempotenceConfigs(final Map<String, Object> final Map<String, Object> originalConfigs = this.originals(); final String acksStr = parseAcks(this.getString(ACKS_CONFIG)); configs.put(ACKS_CONFIG, acksStr); - - // For idempotence producers, values for `RETRIES_CONFIG` and `ACKS_CONFIG` need validation - if (idempotenceEnabled()) { - boolean userConfiguredRetries = originalConfigs.containsKey(RETRIES_CONFIG); - if (userConfiguredRetries && this.getInt(RETRIES_CONFIG) == 0) { - throw new ConfigException("Must set " + ProducerConfig.RETRIES_CONFIG + " to non-zero when using the idempotent producer."); + final boolean userConfiguredIdempotence = this.originals().containsKey(ENABLE_IDEMPOTENCE_CONFIG); + boolean idempotenceEnabled = this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG); + boolean shouldDisableIdempotence = false; + + // For idempotence producers, values for `retries` and `acks` and `max.in.flight.requests.per.connection` need validation + if (idempotenceEnabled) { + final int retries = this.getInt(RETRIES_CONFIG); + if (retries == 0) { + if (userConfiguredIdempotence) { + throw new ConfigException("Must set " + RETRIES_CONFIG + " to non-zero when using the idempotent producer."); + } + log.info("Idempotence will be disabled because {} is set to 0.", RETRIES_CONFIG, retries); + shouldDisableIdempotence = true; } - boolean userConfiguredAcks = originalConfigs.containsKey(ACKS_CONFIG); final short acks = Short.valueOf(acksStr); - if (userConfiguredAcks && acks != (short) -1) { - throw new ConfigException("Must set " + ACKS_CONFIG + " to all in order to use the idempotent " + + if (acks != (short) -1) { + if (userConfiguredIdempotence) { + throw new ConfigException("Must set " + ACKS_CONFIG + " to all in order to use the idempotent " + "producer. Otherwise we cannot guarantee idempotence."); + } + log.info("Idempotence will be disabled because {} is set to {}, not set to 'all'.", ACKS_CONFIG, acks); + shouldDisableIdempotence = true; } - boolean userConfiguredInflightRequests = originalConfigs.containsKey(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION); - if (userConfiguredInflightRequests && MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_FOR_IDEMPOTENCE < this.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION)) { - throw new ConfigException("Must set " + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to at most 5" + + final int inFlightConnection = this.getInt(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION); + if (MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_FOR_IDEMPOTENCE < inFlightConnection) { + if (userConfiguredIdempotence) { + throw new ConfigException("Must set " + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to at most 5" + " to use the idempotent producer."); + } + log.warn("Idempotence will be disabled because {} is set to {}, which is greater than 5. " + + "Please note that in v4.0.0 and onward, this will become an error.", MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, inFlightConnection); + shouldDisableIdempotence = true; } } + + if (shouldDisableIdempotence) { + configs.put(ENABLE_IDEMPOTENCE_CONFIG, false); + } + + // validate `transaction.id` after validating idempotence dependant configs because `enable.idempotence` config might be overridden + idempotenceEnabled = idempotenceEnabled && !shouldDisableIdempotence; Review comment: Good suggestion! Updated. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org