Skip to content

Conversation

frankvicky
Copy link
Contributor

JIRA: KAFKA-9366

Committer Checklist (excluded from commit message)

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

@github-actions github-actions bot added streams core Kafka Broker tools connect kraft mirror-maker-2 storage Pull requests that target the storage module build Gradle build or GitHub Actions docker Official Docker image clients labels Oct 4, 2024
@frankvicky frankvicky marked this pull request as ready for review October 4, 2024 16:42
@frankvicky
Copy link
Contributor Author

This is the initial version. I'd like to run it on CI first.

@frankvicky frankvicky added the do-not-merge PRs that are only open temporarily and should not be merged label Oct 4, 2024
Copy link
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

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

Thanks for tackling this @frankvicky.

Is it really necessary to rename the log4j config files? The KIP mentions a system property that allows us to load log4j.properties files https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158870552#KIP653:Upgradelog4jtolog4j2-Compatibility,Deprecation,andMigrationPlan

I could understand migrating our internal log4j configs for tests and such, but I think we should avoid forcing a change in our production configs.

I also see that this page mentions a log4j to log4j2 bridge https://logging.apache.org/log4j/2.x/migrate-from-log4j1.html. Would that help us at all?

build.gradle Outdated
Comment on lines 2540 to 2544
testRuntimeOnly libs.slf4jLog4j2
testRuntimeOnly libs.junitPlatformLanucher
Copy link
Member

Choose a reason for hiding this comment

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

These (and other similar testRuntimeOnly) should be put into the runtimeTestLibs definition

build.gradle Outdated
implementation libs.slf4jApi
implementation libs.jacksonAnnotations
implementation libs.jacksonDatabind
implementation libs.bndlib
Copy link
Member

Choose a reason for hiding this comment

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

What's this dependency for?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hi @mumrah
I add this to fix the warning during build:

/home/frankvicky/.gradle/caches/modules-2/files-2.1/org.apache.logging.log4j/log4j-api/2.24.1/7ebeb12c20606373005af4232cd0ecca72613dda/log4j-api-2.24.1.jar(/org/apache/logging/log4j/Level.class): warning: Cannot find annotation method 'value()' in type 'BaselineIgnore': class file for aQute.bnd.annotation.baseline.BaselineIgnore not found

Copy link
Member

Choose a reason for hiding this comment

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

I'm not convinced we need that dependency. Also it seems to complain about an annotation so at least we should not need it at runtime, so we should not include it in our distribution package. Currently it's included in the artifact generated by releaseTarGz.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fair enough, I will try to solve this one.

Copy link
Contributor

Choose a reason for hiding this comment

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

The BND annotations are intentionally in the provided Maven scope of all Log4j artifacts, so that these annotations with CLASS retention do not end up in the runtime classpath. You can do the same and add them as compileOnly in Gradle.

The compiler warnings should disappear once JDK-8342833 is fixed.
Untile then we will remove the outdated ones (see apache/logging-log4j2#3133) in the next Log4j release, which should remove the warning on Level.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the information.
I have already changed its scope to compile time.
PTAL 😄

base_dir=$(dirname $0)

if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties"
Copy link
Member

Choose a reason for hiding this comment

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

This could break some existing Kafka installations. If users are extracting in place or copying previous config files to a new installation directory, they will be expecting the log4j.properties to still work.

Copy link
Contributor

Choose a reason for hiding this comment

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

The switch from the legacy to the new configuration format can be based on the presence of specific files:

if [ -f "$base_dir/../config/log4j.properties" ]; then
    echo DEPRECATED: Using Log4j 1.x configuration file \$KAFKA_HOME/config/log4j.properties >&2
    echo To use a Log4j 2.x configuration, create a \$KAFKA_HOME/config/log4j2.xml file and remove the Log4j 1.x configration. >&2
    echo See https://logging.apache.org/log4j/2.x/migrate-from-log4j1.html#Log4j2ConfigurationFormat for details about Log4j configuration file migration. >&2
    export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/log4j.properties"
elif [ -f "$base_dir/../config/log4j2.xml" ]; then
    export KAFKA_LOG4J_OPTS="-Dlog4j2.configurationFile=$base_dir/../config/log4j2.xml"
fi

Copy link
Member

Choose a reason for hiding this comment

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

I’m not sure why we override the KAFKA_LOG4J_OPTS here. We typically allow users to define custom KAFKA_LOG4J_OPTS. Moreover, overriding KAFKA_LOG4J_OPTS can break many end-to-end tests, as they often create log4j configurations dynamically and pass them through KAFKA_LOG4J_OPTS

Noted that we do not require users to strictly use the path $base_dir/../config/log4j2.xml.

@frankvicky frankvicky marked this pull request as draft October 6, 2024 03:19
@frankvicky
Copy link
Contributor Author

Hello @mumrah
Thanks for your feedback. Unfortunately, I barely missed the KIP for some reason, but I'll take a look and adjust the PR accordingly. 😺

@frankvicky frankvicky marked this pull request as ready for review October 10, 2024 13:55
@frankvicky frankvicky marked this pull request as draft October 10, 2024 16:23
@frankvicky frankvicky marked this pull request as ready for review October 11, 2024 11:54
revert unnecessary change
@chia7712
Copy link
Member

@frankvicky Could you please open minor to address following items?

  1. move the deprecation warnings to kafka-run-class KAFKA-9366: Upgrade log4j to log4j2 #17373 (comment)

  2. remove unnecessary reference of jacksonDatabindYaml KAFKA-9366: Upgrade log4j to log4j2 #17373 (comment)

  3. add upgrade notes KAFKA-9366: Upgrade log4j to log4j2 #17373 (comment)

@jolshan
Copy link
Member

jolshan commented Dec 20, 2024

@jolshan thanks for your report. the known issue caused by this PR is the "logger" of connect. And I run the one of test (transactions_upgrade_test.py) on my local:

Interesting. This one fails consistently on our infra (and succeeds without this change). I wonder if there is some different test configuration that causes the issue.

jolshan added a commit to jolshan/kafka that referenced this pull request Dec 20, 2024
@chia7712
Copy link
Member

There are many tests which are failed quickly.

[DEBUG - 2024-12-18 22:36:58,718 - remoteaccount - _log - lineno:180]: ubuntu@worker47: Running ssh command: tail -c +1 /mnt/kafka/server-start-stdout-stderr.log | grep 'Kafka\s*Server.*started'
[DEBUG - 2024-12-18 22:36:58,974 - jmx - start_jmx_tool - lineno:56]: ubuntu@worker47: Not starting jmx tool because no jmx objects are defined
[DEBUG - 2024-12-18 22:36:58,974 - remoteaccount - _log - lineno:180]: ubuntu@worker47: Running ssh command: ps ax | grep -i kafka.Kafka | grep -v grep | awk '{print $1}'
[INFO  - 2024-12-18 22:36:58,983 - runner_client - log - lineno:459]: RunnerClient: kafkatest.tests.core.transactions_upgrade_test.TransactionsUpgradeTest.test_transactions_upgrade.from_kafka_version=3.1.2.metadata_quorum=ISOLATED_KRAFT.use_new_coordinator=False.group_protocol=None: Tearing down...

It can pass the check of grep 'Kafka\s*Server.*started' (kraft controller) but the check of process id fails. I open a MINOR to add more log (#18286) @jolshan Could you please run the patch on your infra?

@jolshan
Copy link
Member

jolshan commented Dec 20, 2024

@chia7712 sure. I will run it and post the results

@jolshan
Copy link
Member

jolshan commented Dec 20, 2024

@frankvicky frankvicky mentioned this pull request Dec 21, 2024
3 tasks
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <[email protected]>

Reviewers: Luke Chen <[email protected]>, Mickael Maison <[email protected]>, Chia-Ping Tsai <[email protected]>
@trnguyencflt
Copy link

trnguyencflt commented Jan 8, 2025

This PR breaks downstream project that depends on kafka_2.13 and reload4j because there is clashing in class LoggingEvent, which exists in reload4j and log4j-1.2-api jars. The application will crash with this exception

Encountered fatal fault: caught exception
java.lang.IncompatibleClassChangeError: class org.apache.log4j.bridge.LogEventAdapter overrides final method org.apache.log4j.spi.LoggingEvent.getProperty(Ljava/lang/String;)Ljava/lang/String;
	at java.base/java.lang.ClassLoader.defineClass1(Native Method)
	at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1027)
	at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:150)
	at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:862)
	at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:760)
	at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:681)
	at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:639)
	at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:188)
	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:526)
	at org.apache.log4j.builders.BuilderManager.<clinit>(BuilderManager.java:59)
	at org.apache.log4j.config.Log4j1Configuration.<init>(Log4j1Configuration.java:46)
	at org.apache.log4j.config.PropertiesConfiguration.<init>(PropertiesConfiguration.java:89)
	at org.apache.log4j.config.PropertiesConfigurationFactory.getConfiguration(PropertiesConfigurationFactory.java:58)
	at org.apache.logging.log4j.core.config.ConfigurationFactory$Factory.getConfiguration(ConfigurationFactory.java:499)
	at org.apache.logging.log4j.core.config.ConfigurationFactory$Factory.getConfiguration(ConfigurationFactory.java:404)
	at org.apache.logging.log4j.core.config.ConfigurationFactory.getConfiguration(ConfigurationFactory.java:318)
	at org.apache.logging.log4j.core.LoggerContext.reconfigure(LoggerContext.java:690)
	at org.apache.logging.log4j.core.LoggerContext.reconfigure(LoggerContext.java:711)
	at org.apache.logging.log4j.core.LoggerContext.start(LoggerContext.java:253)
	at org.apache.logging.log4j.core.impl.Log4jContextFactory.getContext(Log4jContextFactory.java:155)
	at org.apache.logging.log4j.core.impl.Log4jContextFactory.getContext(Log4jContextFactory.java:47)
	at org.apache.logging.log4j.LogManager.getContext(LogManager.java:196)
	at org.apache.logging.log4j.spi.AbstractLoggerAdapter.getContext(AbstractLoggerAdapter.java:137)
	at org.apache.commons.logging.impl.Log4jApiLogFactory$LogAdapter.getContext(Log4jApiLogFactory.java:161)
	at org.apache.logging.log4j.spi.AbstractLoggerAdapter.getLogger(AbstractLoggerAdapter.java:47)
	at org.apache.commons.logging.impl.Log4jApiLogFactory.getInstance(Log4jApiLogFactory.java:210)
	at org.apache.commons.logging.impl.Log4jApiLogFactory.getInstance(Log4jApiLogFactory.java:205)
	at org.apache.commons.logging.LogFactory.getLog(LogFactory.java:987)

@ppkarwasz
Copy link
Contributor

This PR breaks downstream project that depends on kafka_2.13 and reload4j because there is clashing in class LoggingEvent, which exists in reload4j and log4j-1.2-api jars.

You can not have both reload4j and log4j-1.2-api on your classpath, since they are both replacements of log4j:log4j.

@ijuma
Copy link
Member

ijuma commented Jan 8, 2025

The point is that kafka was including reload4j before and hence many other projects aligned with that. It's one thing to cause logging to change, it's another to cause projects not to start anymore with a IncompatibleClassChangeError - that's a much bigger deal.

@chia7712
Copy link
Member

chia7712 commented Jan 8, 2025

This PR breaks downstream project that depends on kafka_2.13 and reload4j

I assume the scenario you describe is to add "kafka_2.13" as dependency for the downstream project. The log4j2 are declared as "implementation" in core module, so downstream project should not "include" log4j2 in the dependencies automatically. not sure why log4j-1.2-api is included in the downstream project. @trnguyencflt Do you add log4j-1.2-api to your project manually?

@chia7712
Copy link
Member

chia7712 commented Jan 8, 2025

@trnguyencflt thanks for your report. I'd like to reduce the gap of upgrading to kafka 4.0 as much as possible, so please share the details to me

@trnguyencflt
Copy link

@chia7712 we include kafka_2.13 as dependency via maven https://github.com/confluentinc/kafka-rest/blob/d6405f714e0e453d6eb5b9a496faac4202c89cab/kafka-rest/pom.xml#L42, and it brings log4j-1.2-api as a runtime dependency (below is from mvn dependency:tree)

[INFO] |  +- org.apache.logging.log4j:log4j-1.2-api:jar:2.24.1:runtime

@chia7712
Copy link
Member

chia7712 commented Jan 9, 2025

@trnguyencflt thanks for your response. log4j-1.2-api is used to convert log4j.properties at runtime, and hence maybe we can remove it from gradle runtime scope and then add it into distribution directly. With that change, log4j-1.2-api gets removed from published pom file but it remains in the distribution.

this is the pom file with above approach and there is not log4j-1.2-api

chia7712@chia7712-ubuntu:~/.m2/repository/org/apache/kafka/kafka_2.13/4.1.0-SNAPSHOT$ cat kafka_2.13-4.1.0-SNAPSHOT.pom | grep log4j
      <groupId>org.apache.logging.log4j</groupId>
      <artifactId>log4j-slf4j-impl</artifactId>
      <groupId>org.apache.logging.log4j</groupId>
      <artifactId>log4j-api</artifactId>
      <groupId>org.apache.logging.log4j</groupId>
      <artifactId>log4j-core</artifactId>

By contrast, log4j-1.2-api is still included by distribution.

chia7712@chia7712-ubuntu:~/project/kafka/core/build/distributions/kafka_2.13-4.1.0-SNAPSHOT$ ls libs | grep log4j
log4j-1.2-api-2.24.1.jar
log4j-api-2.24.1.jar
log4j-core-2.24.1.jar
log4j-slf4j-impl-2.24.1.jar

@frankvicky
Copy link
Contributor Author

Hi @trnguyencflt
Thanks for report.
I have filed KAFKA-18466 to track this issue.
I will filed a PR for this issue soon.

@chia7712
Copy link
Member

@trnguyencflt we have resolved the issue in #18472. If you have some free time, could you please test it on your project?
Please don't hesitate to provide any feedback if you think something has been overlooked. Thank you!

frankvicky added a commit to frankvicky/kafka that referenced this pull request Feb 12, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
frankvicky added a commit to frankvicky/kafka that referenced this pull request Feb 14, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
frankvicky added a commit to frankvicky/kafka that referenced this pull request Feb 17, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
frankvicky added a commit to frankvicky/kafka that referenced this pull request Feb 21, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
frankvicky added a commit to frankvicky/kafka that referenced this pull request Apr 17, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
frankvicky added a commit to frankvicky/kafka that referenced this pull request Apr 17, 2025
…ing kafka

JIRA: KAFKA-18752
In KAFKA-18466 we remove the slf4j impl from runtime scope to fix the
compatible issue (see
apache#17373 (comment)).
However, that also causes a side effect that in running some tasks slf4k
can't find any provider as they are included to "release-only"
We could fix this by adding releasOnly to class path
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

build Gradle build or GitHub Actions ci-approved clients connect consumer core Kafka Broker dependencies Pull requests that update a dependency file docker Official Docker image kraft mirror-maker-2 producer storage Pull requests that target the storage module streams tools

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants