Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-13598: enable idempotence producer by default and validate the configs #11691

Merged
merged 17 commits into from
Feb 5, 2022

Conversation

showuon
Copy link
Contributor

@showuon showuon commented Jan 19, 2022

In short, after v3.0, the default producer is still not idempotence enabled.

In v3.0, we make enable.idempotence default to true, but we didn't adjust the validator and the idempotence enabled check method, so that if user didn't explicitly enable idempotence, this feature won't be turned on. In this PR, I did:

  1. fix the ProducerConfig#idempotenceEnabled method, to make it correctly detect if idempotence is enabled or not
  2. remove some unnecessary config overridden and checks due to we already default acks, retries and enable.idempotence configs.
  3. move the config validator for idempotence producer from KafkaProducer into ProducerConfig. The config validation should be the responsibility of ProducerConfig class.
  4. add an AbstractConfig#hasKeyInOriginals method, to avoid originals configs get copied and only want to check the existence of the key.
  5. fix many broken tests. As mentioned, we didn't actually enable idempotence in v3.0. After this PR, there are some tests broken due to some different behavior between idempotence and non-idempotence producer. Fix them
  6. add tests

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Comment on lines 197 to 198
if ((!config.originals().containsKey(RECONNECT_BACKOFF_MAX_MS_CONFIG)) &&
config.originals().containsKey(RECONNECT_BACKOFF_MS_CONFIG)) {
if ((!config.hasKeyInOriginals(RECONNECT_BACKOFF_MAX_MS_CONFIG)) &&
config.hasKeyInOriginals(RECONNECT_BACKOFF_MS_CONFIG)) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. refactor the originals().containsKey usage. In originals(), we'll make a copy of the configs, and in most cases, we only want to check containsKey. Refactor it by directly check the key in AbstractConfigs to avoid unnecessary map copy.

Comment on lines -517 to -522
final boolean userConfiguredIdempotence = config.originals().containsKey(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG);
final boolean userConfiguredTransactions = config.originals().containsKey(ProducerConfig.TRANSACTIONAL_ID_CONFIG);
if (userConfiguredTransactions && !userConfiguredIdempotence)
log.info("Overriding the default {} to true since {} is specified.", ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG,
ProducerConfig.TRANSACTIONAL_ID_CONFIG);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We won't overriding the default ENABLE_IDEMPOTENCE_CONFIG because we already default to true. No need to inform users.

Comment on lines -543 to -564
private static int configureInflightRequests(ProducerConfig config) {
if (config.idempotenceEnabled() && 5 < config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION)) {
throw new ConfigException("Must set " + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to at most 5" +
" to use the idempotent producer.");
}
return config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION);
}

private static short configureAcks(ProducerConfig config, Logger log) {
boolean userConfiguredAcks = config.originals().containsKey(ProducerConfig.ACKS_CONFIG);
short acks = Short.parseShort(config.getString(ProducerConfig.ACKS_CONFIG));

if (config.idempotenceEnabled()) {
if (!userConfiguredAcks)
log.info("Overriding the default {} to all since idempotence is enabled.", ProducerConfig.ACKS_CONFIG);
else if (acks != -1)
throw new ConfigException("Must set " + ProducerConfig.ACKS_CONFIG + " to all in order to use the idempotent " +
"producer. Otherwise we cannot guarantee idempotence.");
}
return acks;
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this idempotence configs validation into ProducerConfig#postProcessParsedConfig.

Comment on lines -459 to -466
private void maybeOverrideEnableIdempotence(final Map<String, Object> configs) {
boolean userConfiguredIdempotence = this.originals().containsKey(ENABLE_IDEMPOTENCE_CONFIG);
boolean userConfiguredTransactions = this.originals().containsKey(TRANSACTIONAL_ID_CONFIG);

if (userConfiguredTransactions && !userConfiguredIdempotence) {
configs.put(ENABLE_IDEMPOTENCE_CONFIG, true);
}
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unnecessary method since ENABLE_IDEMPOTENCE_CONFIG is default to true now.

}

private void maybeOverrideAcksAndRetries(final Map<String, Object> configs) {
private void postProcessAndValidateIdempotenceConfigs(final Map<String, Object> configs) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We won't override any configs now, change the method name.

Comment on lines 476 to 480
boolean userConfiguredInflightRequests = hasKeyInOriginals(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION);
if (userConfiguredInflightRequests && 5 < 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" +
" to use the idempotent producer.");
}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add validation for MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION here

Comment on lines 519 to 515
boolean idempotenceEnabled = userConfiguredIdempotence && this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG);

if (!idempotenceEnabled && userConfiguredIdempotence && userConfiguredTransactions)
boolean userConfiguredIdempotence = this.hasKeyInOriginals(ENABLE_IDEMPOTENCE_CONFIG);
boolean userConfiguredTransactions = this.hasKeyInOriginals(TRANSACTIONAL_ID_CONFIG);
boolean idempotenceEnabled = !userConfiguredIdempotence || this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the key fix: before the PR, we only considered the idempotenceEnabled when user explicitly set the config and value to true:

 boolean idempotenceEnabled = userConfiguredIdempotence && this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG);

Now, change it to fix the issue.

boolean idempotenceEnabled = !userConfiguredIdempotence || this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG);

@showuon showuon changed the title KAFKA-13598: enable and validate config for idempotence producer KAFKA-13598: enable idempotence producer by default and validate the configs Jan 22, 2022
@showuon
Copy link
Contributor Author

showuon commented Jan 22, 2022

@hachikuji @vvcephei , @ijuma , please take a look. Thanks.

Copy link
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @showuon for this nice cleanup!
I left a few questions and suggestions

@@ -226,6 +227,11 @@ public Password getPassword(String key) {
return keys;
}

public boolean hasKeyInOriginals(String configKey) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about originalsContainsKey()?

@@ -226,6 +227,11 @@ public Password getPassword(String key) {
return keys;
}

public boolean hasKeyInOriginals(String configKey) {
Objects.requireNonNull(configKey, "config key cannot be null");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need this null check? Shouldn't we rely on the behaviour of the underlying Map?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Make sense! We can rely on the underlying map.

import java.util.TreeMap;
import java.util.ArrayList;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's keep the current import order

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Weird, not sure why intellij "help" me reorder it. Updated!

@@ -240,6 +242,7 @@ public void testAcksAndIdempotenceForIdempotentProducers() {
Properties invalidProps2 = new Properties() {{
putAll(baseProps);
setProperty(ProducerConfig.ACKS_CONFIG, "1");
// explicitly enable idempotence should still throw exception
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

enable -> enabling

ProducerMetadata metadata = mock(ProducerMetadata.class);

// Return empty cluster 4 times and cluster from then on
when(metadata.fetch()).thenReturn(emptyCluster, emptyCluster, emptyCluster, emptyCluster, onePartitionCluster);
if (isIdempotenceEnabled) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we change the comment above?

@@ -170,6 +170,7 @@ public void configure(final WorkerConfig config) {
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
producerProps.put(ProducerConfig.RETRIES_CONFIG, 0); // we handle retries in this class
producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false); // disable idempotence since retries is force to 0
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah :( I assumed Connect was benefiting from idempotency automatically, it looks like it isn't the case

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it might be another PR (or KIP) to enable it in Connect.

@@ -2383,6 +2384,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {

private def buildTransactionalProducer(): KafkaProducer[Array[Byte], Array[Byte]] = {
producerConfig.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId)
producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to explicitly set it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, because at the beginning of the AuthorizerIntegrationTest test suite, the idempotence is disabled.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You're right, I missed that!

@@ -143,6 +143,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
val adminClients = Buffer[Admin]()

producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1")
producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we instead drop acks=1? I don't think it is important in this test.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you miss this comment?

Copy link
Contributor Author

@showuon showuon Jan 27, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I forgot to reply this comment. No we should not remove the acks=1 config to enable the idempotence producer here. In this AuthorizerIntegrationTest test suite, we tested normal producer (created here), and IdempotentProducer (created in buildIdempotentProducer method), and transactional producer (created in buildTransactionalProducer method), so if we remove acks=1 here, we won't be able to test normal producer cases.

@@ -1674,6 +1674,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
producerProps.put(ProducerConfig.RETRIES_CONFIG, _retries.toString)
producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, _deliveryTimeoutMs.toString)
producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, _requestTimeoutMs.toString)
// disable the idempotence since some tests want to test the cases when retries=0, and these tests are not testing producers
producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false.toString)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's use "false"

setReadAndWriteAcls(tp2)
sendRecords(producer, numRecords, tp2)
// in idempotence producer, we need to create another producer because the previous one is in FATEL_ERROR state (due to authorization error)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FATEL_ERROR -> FATAL_ERROR
Do we really expect the idempotent producer to not be usable anymore?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that's how it is designed. I added more comment for it to make it clear:
// // in idempotence producer, we need to create another producer because the previous one is in FATEL_ERROR state (due to authorization error)
// If the transaction state in FATEL_ERROR, it'll never transit to other state. check TransactionManager#isTransitionValid for detail

Thank you.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's still fix the typo, it should be FATAL_ERROR

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah, you're right, I forgot it.

@showuon
Copy link
Contributor Author

showuon commented Jan 27, 2022

I found there are flaky tests introduced by this PR. I'll fix them first. Thanks.

@showuon
Copy link
Contributor Author

showuon commented Jan 28, 2022

@mimaison , I've updated the PR and fix the flaky tests. The tests will fail because we have 2 threads: Producer thread and Sender thread in producer. And before the latest commit, I have a bad assumption that the sender thread will run after producer send, so the called times verification will be passed. (ex: verify(metadata, times(5)).fetch();)

I fixed it by removing the the need to fetch metadata in sender thread, so that we can focus on testing the producer behavior as we expected.
Thanks.

@ijuma
Copy link
Contributor

ijuma commented Jan 28, 2022

Are you saying that the change to enable idempotence by default in 3.0 was broken? And we shipped 3.1 with the same critical bug?

Copy link

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @showuon for the patch. Left a couple comments.


if (!idempotenceEnabled && userConfiguredIdempotence && userConfiguredTransactions)
boolean userConfiguredIdempotence = this.originalsContainsKey(ENABLE_IDEMPOTENCE_CONFIG);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this can be simplified a little bit more by getting rid of this variable. How about this?

boolean userConfiguredTransactions = this.originalsContainsKey(TRANSACTIONAL_ID_CONFIG);
boolean idempotenceEnabled = this.getBoolean(ENABLE_IDEMPOTENCE_CONFIG);
if (!idempotenceEnabled && userConfiguredTransactions) {
  throw new ConfigException("Cannot set a " + ProducerConfig.TRANSACTIONAL_ID_CONFIG + " without also enabling idempotence.");
}
return idempotenceEnabled;

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good suggestion! I was affected by the original implementation. Updated.

@@ -226,6 +227,10 @@ public Password getPassword(String key) {
return keys;
}

public boolean originalsContainsKey(String configKey) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AbstractConfig is sadly a public class. Can we do the PR without this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oops! OK, reverted.

@hachikuji
Copy link

hachikuji commented Jan 28, 2022

Are you saying that the change to enable idempotence by default in 3.0 was broken? And we shipped 3.1 with the same critical bug?

@ijuma Sadly so. I think the configuration would even have displayed enable.idempotence=true when printed, but the check disables it unless the user had explicitly provided it.

@mimaison
Copy link
Member

With this change, if a user explicitly sets acks to 0 or 1, we're throwing an error. Currently in order to change acks, users also have to set enable.idempotence=false. I'm expecting many existing applications are currently overriding acks.

So I wonder if instead we should disable idempotency when acks is overridden.

@showuon
Copy link
Contributor Author

showuon commented Feb 16, 2022

@mimaison , do you mean:
So, do you mean, we have some different cases:

  1. if user doesn't set enable.idempotence explicitly: the idempotent producer is only enabled when the acks=all, otherwise, disable idempotence
    ex: a. enable.idempotence unset && acks unset => enable idempotence (this was the intent behind the 3.0 change I think)
    b. enable.idempotence unset && acks=all => enable idempotence (obviously)
    c. enable.idempotence unset && acks!=all => disable idempotence (the case we tried to protect)

  2. if user enable/disable enable.idempotence explicitly, we follow current config validation to throw exceptions when bad config set,
    ex: a. enable.idempotence=true && acks=all => enable idempotence (obviously)
    b. enable.idempotence=false => disable idempotence
    c. enable.idempotence=true && acks!=all => throw exception

And we only do some special check for acks because many users would change this config. For retries and max.in.flight.requests.per.connection, we still follow current config validation no matter enable.idempotence is set explicitly or not.

Does this what you mean?
If so, I'm +1. Actually, if the retries and max.in.flight.requests.per.connection also have the same special check, I also think it makes sense. The purpose is try not to break existing producers after upgrade, which is good.

@mimaison
Copy link
Member

I too think it makes sense to apply the same logic to retries and max.in.flight.requests.per.connection. When disabling idempotency due to a conflicting config, we should log something.

If we decide to make this change, we should also update the docs of these configs to mention this behavior.

@showuon
Copy link
Contributor Author

showuon commented Feb 17, 2022

@ijuma @hachikuji @vvcephei @kirktrue , any thoughts about this suggestion? This way, we don't need #11767 #11769 to do any change.

@vvcephei
Copy link
Contributor

I have to admit that I find all these breakages that are cropping up to be a bit disturbing. I like the idea to default to stronger guarantees, but if the cost is that a lot of existing applications just crash on upgrade, it's not a great tradeoff.

The above logic is fairly complex, but it seems warranted in this case, so I'm +1 as well.

@showuon
Copy link
Contributor Author

showuon commented Feb 17, 2022

KAFKA-13673 is created for this proposal.

@ijuma
Copy link
Contributor

ijuma commented Feb 17, 2022

I agree with the suggestions here. Especially since the idempotence change did not land in 3.0.0 as originally intended. We can continue the discussion in #11788.

rajinisivaram pushed a commit to confluentinc/kafka that referenced this pull request Mar 3, 2022
…configs (apache#11691)

In v3.0, we changed the default value for `enable.idempotence` to true, but we didn't adjust the validator and the `idempotence` enabled check method. So if a user didn't explicitly enable idempotence, this feature won't be turned on. This patch addresses the problem, cleans up associated logic, and fixes tests that broke as a result of properly applying the new default. Specifically it does the following:

1. fix the `ProducerConfig#idempotenceEnabled` method, to make it correctly detect if `idempotence` is enabled or not
2. remove some unnecessary config overridden and checks due to we already default `acks`, `retries` and `enable.idempotence` configs.
3. move the config validator for the idempotent producer from `KafkaProducer` into `ProducerConfig`. The config validation should be the responsibility of `ProducerConfig` class.
4. add an `AbstractConfig#hasKeyInOriginals` method, to avoid `originals` configs get copied and only want to check the existence of the key.
5. fix many broken tests. As mentioned, we didn't actually enable idempotence in v3.0. After this PR, there are some tests broken due to some different behavior between idempotent and non-idempotent producer.
6. add additional tests to validate configuration behavior

Reviewers: Kirk True <kirk@mustardgrain.com>, Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mimaison@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
rajinisivaram pushed a commit to confluentinc/kafka that referenced this pull request Mar 3, 2022
Remove 3.2 notes which were accidentally included after cherry-picking apache#11691. Add the section about the idempotence bug in 3.1 notable items.

Reviewers: Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
@rondagostino
Copy link
Contributor

This PR broke this system test:

TC_PATHS="tests/kafkatest/tests/tools/log4j_appender_test.py" bash tests/docker/run_tests.sh

@showuon
Copy link
Contributor Author

showuon commented Mar 22, 2022

@rondagostino , yes, we are aware of it, and that's why we have #11788 to disable idempotent producer when there are config conflicts. Could you please run again with the build including #11788 change? Thanks.

@rondagostino
Copy link
Contributor

@showuon It still fails at that commit.
git checkout 7c280c1d5f6267acbbebd10d3e58ea4b8fe7a4ef
HEAD is now at 7c280c1d5f KAFKA-13673: disable idempotence when config conflicts (#11788)
./gradlew clean
_DUCKTAPE_OPTIONS="--debug" TC_PATHS="tests/kafkatest/tests/tools/log4j_appender_test.py" bash tests/docker/run_tests.sh
Test fails

@showuon
Copy link
Contributor Author

showuon commented Mar 22, 2022

@rondagostino , could I get the error logs? Thanks.

@rondagostino
Copy link
Contributor

Here is kafka_log4j_appender.log.

SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in [jar:file:/opt/kafka-dev/core/build/dependant-libs-2.13.6/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/tools/build/dependant-libs-2.13.6/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/trogdor/build/dependant-libs-2.13.6/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/api/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/transforms/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/runtime/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/file/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/mirror/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/mirror-client/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/json/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/opt/kafka-dev/connect/basic-auth-extension/build/dependant-libs/slf4j-log4j12-1.7.32.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]
log4j:WARN No appenders could be found for logger (org.apache.kafka.clients.producer.ProducerConfig).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info.

Note this line, which seems suspicious (though I am unfamiliar with this tools/what it does/how it does it, so maybe it is irrelevant).

log4j:WARN No appenders could be found for logger (org.apache.kafka.clients.producer.ProducerConfig).

@rondagostino
Copy link
Contributor

Test passes with this change, but I don't understand why.
git diff log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java

diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
index 23272a2cb5..edaf6589a7 100644
--- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
+++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
@@ -43,6 +43,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG;
 import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG;
 import static org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG;
 import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG;
 import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG;
 import static org.apache.kafka.common.config.SaslConfigs.SASL_KERBEROS_SERVICE_NAME;
 import static org.apache.kafka.common.config.SaslConfigs.SASL_MECHANISM;
@@ -332,6 +333,7 @@ public class KafkaLog4jAppender extends AppenderSkeleton {

         props.put(KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
         props.put(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
+        props.put(ENABLE_IDEMPOTENCE_CONFIG, false);
         this.producer = getKafkaProducer(props);
         LogLog.debug("Kafka producer connected to " + brokerList);
         LogLog.debug("Logging for topic: " + topic);

@michalxo
Copy link
Contributor

@showuon (or Yang Yu) found the issue (is it you @showuon ?)
https://issues.apache.org/jira/browse/KAFKA-13761

@showuon
Copy link
Contributor Author

showuon commented Mar 23, 2022

No, I'm not Yang Yu :)
Glad we found out the root cause!

jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Mar 30, 2022
jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Mar 30, 2022
jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Mar 31, 2022
jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Mar 31, 2022
jeffkbkim pushed a commit to confluentinc/kafka that referenced this pull request May 12, 2022
…configs (apache#11691)

In v3.0, we changed the default value for `enable.idempotence` to true, but we didn't adjust the validator and the `idempotence` enabled check method. So if a user didn't explicitly enable idempotence, this feature won't be turned on. This patch addresses the problem, cleans up associated logic, and fixes tests that broke as a result of properly applying the new default. Specifically it does the following:

1. fix the `ProducerConfig#idempotenceEnabled` method, to make it correctly detect if `idempotence` is enabled or not
2. remove some unnecessary config overridden and checks due to we already default `acks`, `retries` and `enable.idempotence` configs.
3. move the config validator for the idempotent producer from `KafkaProducer` into `ProducerConfig`. The config validation should be the responsibility of `ProducerConfig` class.
4. add an `AbstractConfig#hasKeyInOriginals` method, to avoid `originals` configs get copied and only want to check the existence of the key.
5. fix many broken tests. As mentioned, we didn't actually enable idempotence in v3.0. After this PR, there are some tests broken due to some different behavior between idempotent and non-idempotent producer.
6. add additional tests to validate configuration behavior

Reviewers: Kirk True <kirk@mustardgrain.com>, Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mimaison@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
jeffkbkim pushed a commit to confluentinc/kafka that referenced this pull request May 12, 2022
Remove 3.2 notes which were accidentally included after cherry-picking apache#11691. Add the section about the idempotence bug in 3.1 notable items.

Reviewers: Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Nov 3, 2022
jeffkbkim added a commit to confluentinc/kafka that referenced this pull request Nov 7, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants