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-9366: Upgrade log4j to log4j2 #17373

Open
wants to merge 39 commits into
base: trunk
Choose a base branch
from

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
Contributor

@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
Contributor

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
@@ -2596,6 +2626,7 @@ project(':streams') {
implementation libs.slf4jApi
implementation libs.jacksonAnnotations
implementation libs.jacksonDatabind
implementation libs.bndlib
Copy link
Contributor

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.

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 😄

@@ -22,7 +22,7 @@ fi
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
Contributor

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.

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

@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 11, 2024 11:54
revert unnecessary change
Comment on lines +21 to +25
# Console appender (stdout)
appender.stdout.type=Console
appender.stdout.name=STDOUT
appender.stdout.layout.type=PatternLayout
appender.stdout.layout.pattern=[%d] %p %m (%c)%n

Choose a reason for hiding this comment

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

Have you considered switching to a structured configuration format like XML or YAML?
The properties configuration format is not the default one and is not even one of the original ones (it appeared in version 2.4). It has a lot of quirks to make it easier to read, but also harder to understand.

The XML format does not require additional dependencies. YAML only requires jackson-dataformat-yaml that will only take an additional 400 KiB in Kafka's distribution. In YAML the configuration file would look like:

# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements.  See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License.  You may obtain a copy of the License at
#
#    http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

# Unspecified loggers and loggers with additivity=true output to server.log and stdout
# Note that INFO only applies to unspecified loggers, the log level of the child logger is used otherwise
Configuration:
  Properties:
    Property:
      # Fallback if the system property is not set
      - name: "kafka.logs.dir"
        value: "."
      - name: "logPattern"
        value: "[%d] %p %m (%c)%n%ex"

  # Appenders configuration
  # See: https://logging.apache.org/log4j/2.x/manual/appenders.html
  Appenders:
    Console:
      name: STDOUT
      PatternLayout:
        pattern: "${logPattern}"

    RollingFile:
      - name: KafkaAppender
        fileName: "${sys:kafka.logs.dir}/server.log"
        filePattern: "${sys:kafka.logs.dir}/server.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }
      # State Change appender
      - name: StateChangeAppender
        fileName: "${sys:kafka.logs.dir}/state-change.log"
        filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }
      # Request appender
      - name: RequestAppender
        fileName: "${sys:kafka.logs.dir}/kafka-request.log"
        filePattern: "${sys:kafka.logs.dir}/kafka-request.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }
      # Cleaner appender
      - name: CleanerAppender
        fileName: "${sys:kafka.logs.dir}/log-cleaner.log"
        filePattern: "${sys:kafka.logs.dir}/log-cleaner.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }
      # Controller appender
      - name: ControllerAppender
        fileName: "${sys:kafka.logs.dir}/controller.log"
        filePattern: "${sys:kafka.logs.dir}/controller.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }
      # Authorizer appender
      - name: AuthorizerAppender
        fileName: "${sys:kafka.logs.dir}/kafka-authorizer.log"
        filePattern: "${sys:kafka.logs.dir}/kafka-authorizer.log.%d{yyyy-MM-dd-HH}"
        PatternLayout:
          pattern: "${logPattern}"
        TimeBasedTriggeringPolicy: { }

  # Loggers configuration
  # See: https://logging.apache.org/log4j/2.x/manual/configuration.html#configuring-loggers
  Loggers:
    Root:
      level: INFO
      AppenderRef:
        - ref: STDOUT
        - ref: KafkaAppender
    Loggers:
      # Zookeeper logger
      - name: org.apache.zookeeper
        level: INFO
      # Kafka logger
      - name: kafka
        level: INFO
      # Kafka org.apache logger
      - name: org.apache.kafka
        level: INFO
      # Kafka request logger
      - name: kafka.request.logger
        level: WARN
        additivity: false
        AppenderRef:
          ref: RequestAppender
      # Uncomment the lines below and change log4j.logger.kafka.network.RequestChannel$ to TRACE
      # for additional output related to the handling of requests
#      - name: kafka.network.Processor
#        level: TRACE
#        additivity: false
#        AppenderRef:
#          ref: RequestAppender
#      - name: kafka.server.KafkaApis
#        level: TRACE
#        additivity: false
#        AppenderRef:
#          ref: RequestAppender
      # Kafka network RequestChannel$ logger
      - name: kafka.network.RequestChannel$
        level: WARN
        additivity: false
        AppenderRef:
          ref: RequestAppender
      # KRaft mode controller logger
      - name: org.apache.kafka.controller
        level: INFO
        additivity: false
        AppenderRef:
          ref: ControllerAppender
      # ZK mode controller logger
      - name: kafka.controller
        level: TRACE
        additivity: false
        AppenderRef:
          ref: ControllerAppender
      # LogCleaner logger
      - name: kafka.log.LogCleaner
        level: INFO
        additivity: false
        AppenderRef:
          ref: CleanerAppender
      # State change logger
      - name: state.change.logger
        level: INFO
        additivity: false
        AppenderRef:
          ref: StateChangeAppender
      # Authorizer logger
      - name: kafka.authorizer.logger
        level: INFO
        additivity: false
        AppenderRef:
          ref: AuthorizerAppender

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 @ppkarwasz,
Thanks for your feedback! 😄
As you mentioned, the .properties file format indeed has a drawback of understand. I actually struggled when trying to transform the .properties file from log4j1 to log4j2 -- it was really painful to understand its meaning and transform them at same time.

The yml format looks nice and is more readable, but changing the configuration format might require further discussion, especially since it would introduce additional dependencies to the project.
I will file a jira to initiate a discussion on 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.

Copy link
Member

Choose a reason for hiding this comment

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

As long as we still support the old properties format, we can consider switching to a new format if users provide a log4j2 configuration file. I think it's worth starting a thread on the dev list to explain our options and gather some feedback.

Choose a reason for hiding this comment

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

@mimaison,

To be precise, users will always be able to use the configuration format of their choice, regardless of the format adopted by Kafka. The choice of the configuration file format mostly concerns the default configuration files shipped in the *.tar.gz archive. If Kafka ships with a log4j2.properties file, users will feel forced to use that one and that is IMHO a terrible format to work with.

I have opened a thread on dev@kafka to start a discussion about the subject.

PS: There is currently a primitive Log4j1ConfigurationConverter CLI tool that allows users to automatically convert a log4j.properties files into a log4j2.xml file. I am currently working on extending the list of formats that can be automatically converted (cf. apache/logging-log4j2#2080), but I will probably not have time to support the quirky log4j2.properties format.

Copy link
Member

Choose a reason for hiding this comment

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

I understand what you mean. My point was that some users may have built custom log4j.properties files and run clusters with those, and we want that to continue working. For the new log4j2 files, then yes it makes sense to evaluate the different formats.

Thanks for opening a thread, it's very useful to get input from an Apache Logging PMC member to help us make decisions.

Comment on lines +201 to +205
private void setLevel(org.apache.logging.log4j.Logger logger, Level level) {
String loggerName = logger.getName();
LoggerContext context = (LoggerContext) LogManager.getContext(false);
LoggerConfig loggerConfig = context.getConfiguration().getLoggerConfig(loggerName);
Level currentLevel = loggerConfig.getLevel();

Choose a reason for hiding this comment

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

This looks pretty much as a maintenance headache for the Apache Kafka team. What will happen if the user switches logging implementation (at least 3 logging implementations are supported by the Log4j API, see logging implementations)?

It looks to me that you only use this for JMX. If that is the case, Log4j Core provides an out-of-the-box JMX support. You just need to enable it, since JMX is a potential source of security problems and is disabled by default.

If you need to get and set the levels for other reasons, please open a thread on dev@logging. Users like to change logger levels programmatically so often, that we'd better offer an implementation independent API for that.

Copy link
Member

Choose a reason for hiding this comment

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

This is used by Kafka Connect. We have a REST API that allows changing the log level of all instances in a Kafka Connect cluster. See https://cwiki.apache.org/confluence/display/KAFKA/KIP-495%3A+Dynamically+Adjust+Log+Levels+in+Connect for the details.

Choose a reason for hiding this comment

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

We can probably reach a consensus in the Logging PMC to release a new Log4j Configuration API, that you can use to abstract from the internals of the logging implementation (see this thread on dev@logging).

What is the planned release date for Kafka 4.x? If you wait until the end of the year, this class might not be necessary.

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 @ppkarwasz
Currently, AK 4.0 release is scheduled at January 29th 2025.
For further details, you can refer to the release plan:
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+4.0.0

@frankvicky
Copy link
Contributor Author

There are lots of tests fail come out after merging trunk. 😞
I will take a look......

@frankvicky
Copy link
Contributor Author

Hello everyone,

I am having some trouble debugging the new failures. 😢
The root cause seems to be that these failing tests are unable to capture logs correctly (resulting in empty content), which leads to assertion failures.
It appears that these issues are caused by #17615, although these test cases worked fine when using log4j1.

Any feedback or information would be greatly appreciated.
Thank you!

@frankvicky
Copy link
Contributor Author

Update: It seems that the root cause its because the log event could not be captured correctly.
Screenshot from 2024-11-07 23-18-43
Screenshot from 2024-11-07 23-40-15

Comment on lines 127 to 129
public void append(final LogEvent event) {
synchronized (events) {
events.add(event);
Copy link

@ppkarwasz ppkarwasz Nov 7, 2024

Choose a reason for hiding this comment

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

Suggested change
public void append(final LogEvent event) {
synchronized (events) {
events.add(event);
public void append(final LogEvent event) {
synchronized (events) {
events.add(event.toImmutable());

By default log events are mutable and bound to one thread. They are cleared as soon as the logging call exits.
There is a log4j2.enableThreadlocals kill switch that you can store in a log4j2.component.properties on the test classpath, but it is easier to just take an immutable snapshot.

You can also replace LogCaptureAppender with ListAppender from the log4j-core-test artifact. You can set it up with a config like:

<Configuration>
  <Appenders>
    <List name="LIST"/>
  </Appenders>
  <Loggers>
    <Root level="DEBUG">
      <AppenderRef ref="LIST"/>
    </Root>
  </Loggers>
</Configuration>

Copy link
Contributor Author

@frankvicky frankvicky Nov 8, 2024

Choose a reason for hiding this comment

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

Very appreciate!
I have applied it and tested it locally; it works like a charm.
As for replacing LogCaptureAppender, I think it's a great idea.
IMHO, leveraging an existing tool is better than building our own.
I will file a JIRA to initiate a discussion on 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.

@frankvicky
Copy link
Contributor Author

Hello @mimaison,@showuon,

Since this PR modifies a large number of files, particularly build.gradle, it’s highly susceptible to conflicts with other PRs, making it rather exhausting to resolve these conflicts frequently.
It would be helpful if we could merge this PR into trunk sooner, as there aren’t any outstanding issues or points of contention with it. This would also allow us to begin addressing any follow-up issues.

Many thanks.

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

LGTM! Left a minor comment, but we can address it in a separate PR. Thanks for the effort!

Comment on lines 25 to 33
if [ -f "$base_dir/../config/connect-log4j.properties" ]; then
echo DEPRECATED: Using Log4j 1.x configuration file \$KAFKA_HOME/config/connect-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/connect-log4j.properties"
elif [ -f "$base_dir/../config/connect-log4j2.properties" ]; then
export KAFKA_LOG4J_OPTS="-Dlog4j2.configurationFile=$base_dir/../config/connect-log4j2.properties"
elif [ -f "$base_dir/../config/connect-log4j2.xml" ]; then
export KAFKA_LOG4J_OPTS="-Dlog4j2.configurationFile=$base_dir/../config/connect-log4j2.xml"
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: I thought we'll honor log4j2.properties when both log4j2.properties and log4j.properties exist. No?

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.
Yes, we can change the order of the if-case to achieve that. I will update it in a follow-up PR.

Choose a reason for hiding this comment

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

I think that log4j.properties should have a higher priority than log4j2.properties:

  • Fresh installations of Kafka 4.x will only have a log4j2.properties file.
  • If we find a log4j.properties file, it means that it is either an upgraded installation of Kafka or the user copied their customized configuration.

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm... it makes sense. Already, let's keep the log4j.properties as the highest priority. Thanks.

@showuon
Copy link
Contributor

showuon commented Nov 15, 2024

@mimaison @mumrah @ppkarwasz , do you have any other comments? I'll merge it tomorrow if no other comments. 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 for the updates, I made another pass and left a few comments and suggestions.

README.md Outdated

For example, if you want to see more logs for clients project tests, you can modify [the line](https://github.com/apache/kafka/blob/trunk/clients/src/test/resources/log4j.properties#L21) in `clients/src/test/resources/log4j.properties`
For example, if you want to see more logs for clients project tests, you can modify [the line](https://github.com/apache/kafka/blob/trunk/clients/src/test/resources/log4j.properties#L21) in `clients/src/test/resources/log4j2.properties`
Copy link
Member

Choose a reason for hiding this comment

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

Why do you want to do it in a separate PR?
If we merge as is we instruct users to go check clients/src/test/resources/log4j2.properties but instead link to another file. If we update the comment we need to update the link.

if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
if [ -f "$base_dir/../config/connect-log4j.properties" ]; then
echo DEPRECATED: Using Log4j 1.x configuration file \$KAFKA_HOME/config/connect-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
Copy link
Member

Choose a reason for hiding this comment

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

Why do we recommend creating an XML file? Should we point to the migration guide and to the log4j2 example file Kafka will have under config

Copy link
Member

Choose a reason for hiding this comment

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

Same in the other scripts

# See the License for the specific language governing permissions and
# limitations under the License.

name=ConfigConnectConfig
Copy link
Member

Choose a reason for hiding this comment

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

Would ConnectConfig be a better name?

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 looks have a module name prefix is a little bit silly.

Comment on lines 99 to 101
# Zookeeper logger
logger.zookeeper.name=org.apache.zookeeper
logger.zookeeper.level=INFO
Copy link
Member

Choose a reason for hiding this comment

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

We since removed ZooKeeper from the existing log4j properties files in trunk (085b27e), so let's not re-add ZooKeeper stuff to remove it again later.

}

}
}
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 newline

@@ -88,7 +89,7 @@ class SocketServerTest {
var server: SocketServer = _
val sockets = new ArrayBuffer[Socket]

private val kafkaLogger = org.apache.log4j.LogManager.getLogger("kafka")
private val kafkaLogger = org.apache.logging.log4j.LogManager.getLogger("kafka")
Copy link
Member

Choose a reason for hiding this comment

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

Can we import LogManager?

@@ -152,7 +152,9 @@ versions += [
// Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid
zstd: "1.5.6-6",
junitPlatform: "1.10.2",
hdrHistogram: "2.2.2"
hdrHistogram: "2.2.2",
log4j2: "2.24.1",
Copy link
Member

Choose a reason for hiding this comment

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

Nit: I know dependencies are not fully ordered but can we insert it roughly where it should be in the list instead of appending at the end.

@@ -22,4 +22,4 @@ log4j.appender.FILE.ImmediateFlush=true
# Set the append to false, overwrite
log4j.appender.FILE.Append=false
log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
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 new line. Same in a few other files

build.gradle Outdated
implementation libs.log4j2Api
implementation libs.log4j2Core
implementation libs.log4j1Bridge2Api
implementation libs.spotbugs
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 really need this as implementation? This is make it part of our release artifact.

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 will see if we could avoid it be included in release artifact

*/
private static final String ROOT_LOGGER_NAME = "root";
private static final String ROOT_LOGGER_NAME = "";
Copy link
Member

Choose a reason for hiding this comment

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

This effectively changes the behavior of the /admin/loggers endpoint of the Connect REST API.
The endpoints accept the logger name in the path /admin/loggers/{name}. If the root logger is the empty string, it's not possible to query it anymore. I wonder if we should still expose the root logger as root (I assume it's possible to rename it somewhere here or in LoggingResource). cc @gharris1727 WDYT

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 core Kafka Broker docker Official Docker image kraft mirror-maker-2 storage Pull requests that target the storage module streams tools
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants