HADOOP-14549. Use GenericTestUtils.setLogLevel when available in hadoop-tools. Contributed by wenxin he.

This commit is contained in:
Akira Ajisaka 2017-06-26 16:41:00 +09:00
parent 0111711d8b
commit a594f96fb1
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
6 changed files with 39 additions and 33 deletions

View File

@ -134,6 +134,11 @@ public static void setLogLevel(org.slf4j.Logger logger, Level level) {
setLogLevel(toLog4j(logger), level);
}
public static void setLogLevel(org.slf4j.Logger logger,
org.slf4j.event.Level level) {
setLogLevel(toLog4j(logger), Level.toLevel(level.toString()));
}
/**
* Extracts the name of the method where the invocation has happened
* @return String name of the invoking method

View File

@ -28,8 +28,6 @@
import java.util.List;
import java.util.StringTokenizer;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -43,15 +41,18 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.JarFinder;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.Assert;
import static org.junit.Assert.*;
import static org.slf4j.LoggerFactory.getLogger;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.event.Level;
/**
* test {@link HadoopArchives}
@ -62,9 +63,8 @@ public class TestHadoopArchives {
.getJar(HadoopArchives.class);
{
((Log4JLogger) LogFactory.getLog(org.apache.hadoop.security.Groups.class))
.getLogger().setLevel(Level.ERROR);
GenericTestUtils.setLogLevel(
getLogger(org.apache.hadoop.security.Groups.class), Level.ERROR);
}
private static final String inputDir = "input";

View File

@ -24,8 +24,6 @@
import java.util.Arrays;
import java.util.Random;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -40,14 +38,16 @@
import org.apache.hadoop.mapred.MiniMRClientClusterFactory;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.log4j.Level;
import org.slf4j.event.Level;
import static org.slf4j.LoggerFactory.getLogger;
public class TestDistCh extends junit.framework.TestCase {
{
((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.StateChange")
).getLogger().setLevel(Level.ERROR);
GenericTestUtils.setLogLevel(
getLogger("org.apache.hadoop.hdfs.StateChange"), Level.ERROR);
GenericTestUtils.setLogLevel(DataNode.LOG, Level.ERROR);
((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ERROR);
GenericTestUtils.setLogLevel(getLogger(FSNamesystem.class), Level.ERROR);
}
static final Long RANDOM_NUMBER_GENERATOR_SEED = null;

View File

@ -17,19 +17,18 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.hadoop.tools.rumen.JobStoryProducer;
import org.apache.hadoop.util.ExitUtil;
import org.apache.log4j.Level;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.event.Level;
import java.io.ByteArrayOutputStream;
@ -41,6 +40,7 @@
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
import static org.junit.Assert.*;
import static org.slf4j.LoggerFactory.getLogger;
public class TestGridmixSubmission extends CommonJobTest {
private static File inSpace = new File("src" + File.separator + "test"
@ -48,8 +48,8 @@ public class TestGridmixSubmission extends CommonJobTest {
static {
((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
.getLogger().setLevel(Level.DEBUG);
GenericTestUtils.setLogLevel(
getLogger("org.apache.hadoop.mapred.gridmix"), Level.DEBUG);
}

View File

@ -17,28 +17,29 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.log4j.Level;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.event.Level;
import java.io.IOException;
import static org.slf4j.LoggerFactory.getLogger;
/*
Test LoadJob Gridmix sends data to job and after that
*/
public class TestLoadJob extends CommonJobTest {
public static final Log LOG = LogFactory.getLog(Gridmix.class);
public static final Logger LOG = getLogger(Gridmix.class);
static {
((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
.getLogger().setLevel(Level.DEBUG);
((Log4JLogger) LogFactory.getLog(StressJobFactory.class)).getLogger()
.setLevel(Level.DEBUG);
GenericTestUtils.setLogLevel(
getLogger("org.apache.hadoop.mapred.gridmix"), Level.DEBUG);
GenericTestUtils.setLogLevel(
getLogger(StressJobFactory.class), Level.DEBUG);
}

View File

@ -17,32 +17,32 @@
*/
package org.apache.hadoop.mapred.gridmix;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.tools.rumen.JobStory;
import org.apache.log4j.Level;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.event.Level;
import java.io.IOException;
import java.util.List;
import static org.junit.Assert.*;
import static org.slf4j.LoggerFactory.getLogger;
public class TestSleepJob extends CommonJobTest {
public static final Log LOG = LogFactory.getLog(Gridmix.class);
public static final Logger LOG = getLogger(Gridmix.class);
static {
((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
.getLogger().setLevel(Level.DEBUG);
GenericTestUtils.setLogLevel(
getLogger("org.apache.hadoop.mapred.gridmix"), Level.DEBUG);
}
static GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.REPLAY;