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

[FLINK-2703] Prepare Flink for using it with Logback. #1194

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
129 changes: 129 additions & 0 deletions docs/apis/best_practices.md
Original file line number Diff line number Diff line change
Expand Up @@ -269,3 +269,132 @@ For Google Protobuf you need the following Maven dependency:


Please adjust the versions of both libraries as needed.


## Using Logback instead of Log4j

**Note: This tutorial is applicable starting from Flink 0.10**

Apache Flink is using [slf4j](http:https://www.slf4j.org/) as the logging abstraction in the code. Users are advised to use sfl4j as well in their user functions.

Sfl4j is a compile-time logging interface that can use different logging implementations at runtime, such as [log4j](http:https://logging.apache.org/log4j/2.x/) or [Logback](http:https://logback.qos.ch/).

Flink is depending on Log4j by default. This page describes how to use Flink with Logback.

To get a logger instance in the code, use the following code:


{% highlight java %}
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class MyClass implements MapFunction {
private static final Logger LOG = LoggerFactory.getLogger(MyClass.class);
// ...
{% endhighlight %}


### Use Logback when running Flink out of the IDE / from a Java application


In all cases were classes are executed with a classpath created by a dependency manager such as Maven, Flink will pull log4j into the classpath.

Therefore, you will need to exclude log4j from Flink's dependencies. The following description will assume a Maven project created from a [Flink quickstart](../quickstart/java_api_quickstart.html).

Change your projects `pom.xml` file like this:

{% highlight xml %}
<dependencies>
<!-- Add the two required logback dependencies -->
<dependency>
<groupId>ch.qos.logback</groupId>
<artifactId>logback-core</artifactId>
<version>1.1.3</version>
</dependency>
<dependency>
<groupId>ch.qos.logback</groupId>
<artifactId>logback-classic</artifactId>
<version>1.1.3</version>
</dependency>

<!-- Add the log4j -> sfl4j (-> logback) bridge into the classpath
Hadoop is logging to log4j! -->
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>log4j-over-slf4j</artifactId>
<version>1.7.7</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-java</artifactId>
<version>0.10-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-core</artifactId>
<version>0.10-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients</artifactId>
<version>0.10-SNAPSHOT</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
{% endhighlight %}

The following changes were done in the `<dependencies>` section:

* Exclude all `log4j` dependencies from all Flink dependencies: This causes Maven to ignore Flink's transitive dependencies to log4j.
* Exclude the `slf4j-log4j12` artifact from Flink's dependencies: Since we are going to use the slf4j to logback binding, we have to remove the slf4j to log4j binding.
* Add the Logback dependencies: `logback-core` and `logback-classic`
* Add dependencies for `log4j-over-slf4j`. `log4j-over-slf4j` is a tool which allows legacy applications which are directly using the Log4j APIs to use the Slf4j interface. Flink depends on Hadoop which is directly using Log4j for logging. Therefore, we need to redirect all logger calls from Log4j to Slf4j which is in turn logging to Logback.

Please note that you need to manually add the exclusions to all new Flink dependencies you are adding to the pom file.

You may also need to check if other dependencies (non Flink) are pulling in log4j bindings. You can analyze the dependencies of your project with `mvn dependency:tree`.



### Use Logback when running Flink on a cluster

This tutorial is applicable when running Flink on YARN or as a standalone cluster.

In order to use Logback instead of Log4j with Flink, you need to remove the `log4j-1.2.xx.jar` and `sfl4j-log4j12-xxx.jar` from the `lib/` directory.

Next, you need to put the following jar files into the `lib/` folder:

* `logback-classic.jar`
* `logback-core.jar`
* `log4j-over-slf4j.jar`: This bridge needs to be present in the classpath for redirecting logging calls from Hadoop (which is using Log4j) to Slf4j.

2 changes: 2 additions & 0 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,8 @@ under the License.
<exclude>org.apache.flink:flink-scala-examples</exclude>
<exclude>org.apache.flink:flink-streaming-examples</exclude>
<exclude>org.apache.flink:flink-python</exclude>
<exclude>org.slf4j:slf4j-log4j12</exclude>
<exclude>log4j:log4j</exclude>
</excludes>
</artifactSet>
<transformers>
Expand Down
3 changes: 3 additions & 0 deletions flink-dist/src/main/assemblies/bin.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,9 @@ under the License.

<includes>
<include>org.apache.flink:flink-python</include>
<include>org.apache.flink:flink-python</include>
Copy link
Contributor

Choose a reason for hiding this comment

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

Accidental line duplication?

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... Yes. I'll remove that again

<include>org.slf4j:slf4j-log4j12</include>
<include>log4j:log4j</include>
</includes>
</dependencySet>
</dependencySets>
Expand Down
9 changes: 8 additions & 1 deletion flink-shaded-hadoop/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,14 @@ under the License.
<filter>
<artifact>org.slf4j:*</artifact>
<excludes>
<exclude>org/slf4j/impl/StaticLoggerBinder*</exclude>
<exclude>org/slf4j/impl/**</exclude>
</excludes>
</filter>
<!-- Exclude Hadoop's log4j. Hadoop can use Flink's log4j dependency -->
<filter>
<artifact>log4j:*</artifact>
<excludes>
<exclude>org/apache/log4j/**</exclude>
</excludes>
</filter>
</filters>
Expand Down