Skip to content

Commit

Permalink
[FLINK-1086] Replace JCL with SLF4J and Log4j with LOGBack
Browse files Browse the repository at this point in the history
- Excluded Kafka's transitive dependencies: jmxtools and jmxri
- Corrected encoder pattern in logback.xml
- Removed explicit logging access. Loggers are now configured by
  configuration files. Fixed Yarn issue with multiple logging
  bindings.

This closes apache#111.
  • Loading branch information
tillrohrmann authored and uce committed Sep 5, 2014
1 parent 3aa5511 commit 0818850
Show file tree
Hide file tree
Showing 236 changed files with 1,068 additions and 1,227 deletions.
18 changes: 12 additions & 6 deletions DEPENDENCIES
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ under the Apache License (v 2.0):
- Apache Commons FileUpload (http:https://commons.apache.org/fileupload/)
- Apache Commons IO (http:https://commons.apache.org/io/)
- Apache Commons Math (http:https://commons.apache.org/proper/commons-math/)
- Apache Log4J (http:https://logging.apache.org/log4j/1.2/)
- Apache Avro (http:https://avro.apache.org)
- Apache Hadoop (http:https://hadoop.apache.org)
- Apache Derby (http:https://db.apache.org/derby/)
Expand All @@ -43,6 +42,8 @@ The Apache Flink project depends on and/or bundles the following components
under the Eclipse Public License (v 1.0)

- JUnit (http:https://junit.org/)
- LOGback (http:https://logback.qos.ch)
Copyright (C) 1999-2012, QOS.ch. All rights reserved.

You may obtain a copy of the Eclipse Public License (v 1.0) at
https://www.eclipse.org/legal/epl-v10.html
Expand Down Expand Up @@ -188,14 +189,19 @@ Copyright (c) 2008 Alexander Beider & Stephen P. Morse.


-----------------------------------------------------------------------
Apache Log4J
LOGBack
-----------------------------------------------------------------------

ResolverUtil.java
Copyright 2005-2006 Tim Fennell
Copyright (C) 1999-2012, QOS.ch

Dumbster SMTP test server
Copyright 2004 Jason Paul Kitchen
This program and the accompanying materials are dual-licensed under
either the terms of the Eclipse Public License v1.0 as published by
the Eclipse Foundation

or (per the licensee's choosing)

under the terms of the GNU Lesser General Public License version 2.1
as published by the Free Software Foundation.


-----------------------------------------------------------------------
Expand Down
61 changes: 61 additions & 0 deletions docs/internal_logging.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
---
title: "How to use logging"
---

The logging in Flink is implemented using the slf4j logging interface. As underlying logging framework, logback is used.

## Configuring logback

For users and developers alike it is important to control the logging framework.
The configuration of the logging framework is exclusively done by configuration files.
The configuration file either has to be specified by setting the environment property `-Dlogback.configurationFile=<file>` or by putting `logback.xml` in the classpath.
The `conf` directory contains a `logback.xml` file which can be modified and is used if Flink is started outside of an IDE and with the provided starting scripts.
The provided `logback.xml` has the following form:

``` xml
<configuration>
<appender name="file" class="ch.qos.logback.core.FileAppender">
<file>${log.file}</file>
<append>false</append>
<encoder>
<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>

<root level="INFO">
<appender-ref ref="file"/>
</root>
</configuration>
```

In order to control the logging level of `org.apache.flink.runtime.jobgraph.JobGraph`, for example, one would have to add the following line to the configuration file.
``` xml
<logger name="org.apache.flink.runtime.jobgraph.JobGraph" level="DEBUG"/>
```

For further information on configuring logback see [LOGback's manual](http:https://logback.qos.ch/manual/configuration.html).

## Best practices for developers

The loggers using slf4j are created by calling
``` java
import org.slf4j.LoggerFactory
import org.slf4j.Logger

Logger LOG = LoggerFactory.getLogger(Foobar.class)
```

In order to benefit most from slf4j, it is recommended to use its placeholder mechanism.
Using placeholders allows to avoid unnecessary string constructions in case that the logging level is set so high that the message would not be logged.
The syntax of placeholders is the following:
``` java
LOG.info("This message contains {} placeholders. {}", 2, "Yippie");
```

Placeholders can also be used in conjunction with exceptions which shall be logged.

``` java
catch(Exception exception){
LOG.error("An {} occurred.", "error", exception);
}
```
1 change: 1 addition & 0 deletions docs/internal_overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -39,3 +39,4 @@ or pull request that updates these documents as well.*
- [RPC and JobManager Communication](rpc_transfer.html)
-->

- [How-to: Using logging in Flink](internal_logging.html)
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import org.apache.avro.io.DatumReader;
import org.apache.avro.reflect.ReflectDatumReader;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.avro.FSDataInputStreamWrapper;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
Expand All @@ -43,7 +43,7 @@ public class AvroInputFormat<E> extends FileInputFormat<E> implements ResultType

private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(AvroInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(AvroInputFormat.class);


private final Class<E> avroValueType;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import org.apache.avro.io.DatumReader;
import org.apache.avro.reflect.ReflectDatumReader;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.avro.AvroBaseValue;
import org.apache.flink.api.avro.FSDataInputStreamWrapper;
import org.apache.flink.api.java.record.io.FileInputFormat;
Expand All @@ -42,7 +42,7 @@ public class AvroInputFormat<E> extends FileInputFormat {

private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(AvroInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(AvroInputFormat.class);


private final Class<? extends AvroBaseValue<E>> avroWrapperTypeClass;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.avro.FSDataInputStreamWrapper;
import org.apache.flink.api.java.record.io.FileInputFormat;
import org.apache.flink.core.fs.FileInputSplit;
Expand Down Expand Up @@ -64,7 +64,7 @@
public class AvroRecordInputFormat extends FileInputFormat {
private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(AvroRecordInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(AvroRecordInputFormat.class);

private FileReader<GenericRecord> dataFileReader;
private GenericRecord reuseAvroRecord = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.flink.client.program.Client;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.LogUtils;
import org.junit.Assert;
import org.junit.Test;

Expand All @@ -36,10 +35,6 @@ public class AvroExternalJarProgramITCase {

private static final String TEST_DATA_FILE = "/testdata.avro";

static {
LogUtils.initializeDefaultTestConsoleLogger();
}

@Test
public void testExternalProgram() {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,21 +31,17 @@

import org.apache.avro.reflect.ReflectDatumReader;
import org.apache.avro.reflect.ReflectDatumWriter;
import org.apache.flink.api.avro.DataInputDecoder;
import org.apache.flink.api.avro.DataOutputEncoder;
import org.apache.flink.api.java.record.io.avro.generated.Colors;
import org.apache.flink.api.java.record.io.avro.generated.User;
import org.apache.flink.util.StringUtils;
import org.junit.Test;

import static org.junit.Assert.*;


/**
* Tests the {@link DataOutputEncoder} and {@link DataInputDecoder} classes for Avro serialization.
*/
public class EncoderDecoderTest {

@Test
public void testComplexStringsDirecty() {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.flink.api.java.record.io.avro.AvroRecordInputFormat;
import org.apache.flink.api.java.record.io.avro.AvroRecordInputFormat.BooleanListValue;
import org.apache.flink.api.java.record.io.avro.AvroRecordInputFormat.LongMapValue;
import org.apache.flink.api.java.record.io.avro.AvroRecordInputFormat.StringListValue;
Expand Down Expand Up @@ -68,8 +67,7 @@ public class AvroRecordInputFormatTest {
final static long TEST_MAP_VALUE1 = 8546456L;
final static CharSequence TEST_MAP_KEY2 = "KEY 2";
final static long TEST_MAP_VALUE2 = 17554L;



@Before
public void createFiles() throws IOException {
testFile = File.createTempFile("AvroInputFormatTest", null);
Expand Down
29 changes: 29 additions & 0 deletions flink-addons/flink-avro/src/test/resources/logback-test.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
<!--
~ 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:https://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.
-->

<configuration>
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>

<root level="WARN">
<appender-ref ref="STDOUT"/>
</root>
</configuration>
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
import java.io.ObjectOutputStream;
import java.util.ArrayList;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
import org.apache.flink.api.common.io.statistics.BaseStatistics;
Expand All @@ -51,7 +51,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable> implement

private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(HadoopInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(HadoopInputFormat.class);

private org.apache.hadoop.mapred.InputFormat<K, V> mapredInputFormat;
private Class<K> keyClass;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@
import java.util.ArrayList;
import java.util.List;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
import org.apache.flink.api.common.io.statistics.BaseStatistics;
Expand Down Expand Up @@ -54,7 +54,7 @@ public class HadoopInputFormat<K extends Writable, V extends Writable> implement

private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(HadoopInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(HadoopInputFormat.class);

private org.apache.hadoop.mapreduce.InputFormat<K, V> mapreduceInputFormat;
private Class<K> keyClass;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
<!--
~ 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:https://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.
-->

<configuration>
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>

<root level="WARN">
<appender-ref ref="STDOUT"/>
</root>
</configuration>
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@
import java.util.ArrayList;
import java.util.List;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.addons.hbase.common.HBaseKey;
import org.apache.flink.addons.hbase.common.HBaseResult;
import org.apache.flink.addons.hbase.common.HBaseUtil;
Expand All @@ -51,7 +51,7 @@ public class TableInputFormat implements InputFormat<Record, TableInputSplit> {

private static final long serialVersionUID = 1L;

private static final Log LOG = LogFactory.getLog(TableInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(TableInputFormat.class);

/** A handle on an HBase table */
private HTable table;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import java.sql.SQLException;
import java.sql.Statement;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.io.statistics.BaseStatistics;
import org.apache.flink.api.java.tuple.Tuple;
Expand All @@ -48,7 +48,7 @@ public class JDBCInputFormat<OUT extends Tuple> implements InputFormat<OUT, Inpu
private static final long serialVersionUID = 1L;

@SuppressWarnings("unused")
private static final Log LOG = LogFactory.getLog(JDBCInputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class);

private String username;
private String password;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
import java.sql.PreparedStatement;
import java.sql.SQLException;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.Configuration;
Expand All @@ -42,7 +42,7 @@ public class JDBCOutputFormat<OUT extends Tuple> implements OutputFormat<OUT> {
private static final long serialVersionUID = 1L;

@SuppressWarnings("unused")
private static final Log LOG = LogFactory.getLog(JDBCOutputFormat.class);
private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class);

private String username;
private String password;
Expand Down
Loading

0 comments on commit 0818850

Please sign in to comment.