MAPREDUCE-6977 Move logging APIs over to slf4j in hadoop-mapreduce-client-common.

Contributed by Jinjiang Ling.
This commit is contained in:
Steve Loughran 2017-10-27 10:43:40 +01:00
parent 36e158ae98
commit de09716fb8
9 changed files with 41 additions and 39 deletions

View File

@ -39,8 +39,6 @@ import java.util.concurrent.Future;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalDirAllocator;
@ -59,14 +57,16 @@ import org.apache.hadoop.yarn.util.FSDownload;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A helper class for managing the distributed cache for {@link LocalJobRunner}.
*/
@SuppressWarnings("deprecation")
class LocalDistributedCacheManager {
public static final Log LOG =
LogFactory.getLog(LocalDistributedCacheManager.class);
public static final Logger LOG =
LoggerFactory.getLogger(LocalDistributedCacheManager.class);
private List<String> localArchives = new ArrayList<String>();
private List<String> localFiles = new ArrayList<String>();
@ -230,7 +230,7 @@ class LocalDistributedCacheManager {
final URL[] urls = new URL[localClasspaths.size()];
for (int i = 0; i < localClasspaths.size(); ++i) {
urls[i] = new File(localClasspaths.get(i)).toURI().toURL();
LOG.info(urls[i]);
LOG.info(urls[i].toString());
}
return AccessController.doPrivileged(new PrivilegedAction<ClassLoader>() {
@Override

View File

@ -38,8 +38,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import javax.crypto.KeyGenerator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -74,13 +72,15 @@ import org.apache.hadoop.util.ReflectionUtils;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Implements MapReduce locally, in-process, for debugging. */
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class LocalJobRunner implements ClientProtocol {
public static final Log LOG =
LogFactory.getLog(LocalJobRunner.class);
public static final Logger LOG =
LoggerFactory.getLogger(LocalJobRunner.class);
/** The maximum number of map tasks to run in parallel in LocalJobRunner */
public static final String LOCAL_MAX_MAPS =
@ -587,7 +587,7 @@ public class LocalJobRunner implements ClientProtocol {
} else {
this.status.setRunState(JobStatus.FAILED);
}
LOG.warn(id, t);
LOG.warn(id.toString(), t);
JobEndNotifier.localRunnerNotification(job, status);
@ -721,17 +721,17 @@ public class LocalJobRunner implements ClientProtocol {
@Override
public synchronized void fsError(TaskAttemptID taskId, String message)
throws IOException {
LOG.fatal("FSError: "+ message + "from task: " + taskId);
LOG.error("FSError: "+ message + "from task: " + taskId);
}
@Override
public void shuffleError(TaskAttemptID taskId, String message) throws IOException {
LOG.fatal("shuffleError: "+ message + "from task: " + taskId);
LOG.error("shuffleError: "+ message + "from task: " + taskId);
}
public synchronized void fatalError(TaskAttemptID taskId, String msg)
throws IOException {
LOG.fatal("Fatal: "+ msg + "from task: " + taskId);
LOG.error("Fatal: "+ msg + "from task: " + taskId);
}
@Override

View File

@ -24,20 +24,21 @@ import java.net.URLDecoder;
import java.net.URLEncoder;
import static java.nio.charset.StandardCharsets.UTF_8;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class FileNameIndexUtils {
// Sanitize job history file for predictable parsing
static final String DELIMITER = "-";
static final String DELIMITER_ESCAPE = "%2D";
private static final Log LOG = LogFactory.getLog(FileNameIndexUtils.class);
private static final Logger LOG =
LoggerFactory.getLogger(FileNameIndexUtils.class);
// Job history file names need to be backwards compatible
// Only append new elements to the end of this list

View File

@ -28,8 +28,6 @@ import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -48,6 +46,8 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Unstable
@ -121,7 +121,8 @@ public class JobHistoryUtils {
public static final String TIMESTAMP_DIR_REGEX = "\\d{4}" + "\\" + Path.SEPARATOR + "\\d{2}" + "\\" + Path.SEPARATOR + "\\d{2}";
public static final Pattern TIMESTAMP_DIR_PATTERN = Pattern.compile(TIMESTAMP_DIR_REGEX);
private static final String TIMESTAMP_DIR_FORMAT = "%04d" + File.separator + "%02d" + File.separator + "%02d";
private static final Log LOG = LogFactory.getLog(JobHistoryUtils.class);
private static final Logger LOG =
LoggerFactory.getLogger(JobHistoryUtils.class);
private static final PathFilter CONF_FILTER = new PathFilter() {
@Override

View File

@ -21,8 +21,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
@ -38,12 +36,14 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.util.Records;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class MRDelegationTokenRenewer extends TokenRenewer {
private static final Log LOG = LogFactory
.getLog(MRDelegationTokenRenewer.class);
private static final Logger LOG = LoggerFactory
.getLogger(MRDelegationTokenRenewer.class);
@Override
public boolean handleKind(Text kind) {

View File

@ -20,19 +20,19 @@ package org.apache.hadoop.mapreduce.v2.security.client;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenSelector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ClientHSTokenSelector implements
TokenSelector<MRDelegationTokenIdentifier> {
private static final Log LOG = LogFactory
.getLog(ClientHSTokenSelector.class);
private static final Logger LOG = LoggerFactory
.getLogger(ClientHSTokenSelector.class);
@SuppressWarnings("unchecked")
public Token<MRDelegationTokenIdentifier> selectToken(Text service,

View File

@ -34,8 +34,6 @@ import java.util.Map;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -70,6 +68,8 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.Apps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Helper class for MR applications
@ -77,7 +77,7 @@ import org.apache.hadoop.yarn.util.Apps;
@Private
@Unstable
public class MRApps extends Apps {
public static final Log LOG = LogFactory.getLog(MRApps.class);
public static final Logger LOG = LoggerFactory.getLogger(MRApps.class);
public static String toString(JobId jid) {
return jid.toString();

View File

@ -27,8 +27,6 @@ import java.io.InputStreamReader;
import java.util.Random;
import java.util.StringTokenizer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@ -44,11 +42,13 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestLocalModeWithNewApis {
public static final Log LOG =
LogFactory.getLog(TestLocalModeWithNewApis.class);
public static final Logger LOG =
LoggerFactory.getLogger(TestLocalModeWithNewApis.class);
Configuration conf;

View File

@ -31,8 +31,6 @@ import org.junit.Assert;
import org.junit.Test;
import static org.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.filecache.DistributedCache;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -50,6 +48,8 @@ import org.apache.hadoop.mapreduce.TaskInputOutputContext;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Tests the use of the
* {@link org.apache.hadoop.mapreduce.filecache.DistributedCache} within the
@ -81,8 +81,8 @@ public class TestMRWithDistributedCache {
}
}
private static final Log LOG =
LogFactory.getLog(TestMRWithDistributedCache.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestMRWithDistributedCache.class);
private static class DistributedCacheChecker {