HADOOP-14624. Add GenericTestUtils.DelayAnswer that accept slf4j logger API. Contributed by Ian Pickering and Wenxin He.

This commit is contained in:
Giovanni Matteo Fumarola 2018-08-17 14:40:00 -07:00
parent ab37423ad8
commit 79c97f6a0b
13 changed files with 59 additions and 40 deletions

View File

@ -501,7 +501,7 @@ public void clearOutput() {
* method is called, then waits on another before continuing.
*/
public static class DelayAnswer implements Answer<Object> {
private final Log LOG;
private final org.slf4j.Logger LOG;
private final CountDownLatch fireLatch = new CountDownLatch(1);
private final CountDownLatch waitLatch = new CountDownLatch(1);
@ -514,7 +514,7 @@ public static class DelayAnswer implements Answer<Object> {
private volatile Throwable thrown;
private volatile Object returnValue;
public DelayAnswer(Log log) {
public DelayAnswer(org.slf4j.Logger log) {
this.LOG = log;
}
@ -611,13 +611,13 @@ public int getResultCount() {
*/
public static class DelegateAnswer implements Answer<Object> {
private final Object delegate;
private final Log log;
private final org.slf4j.Logger log;
public DelegateAnswer(Object delegate) {
this(null, delegate);
}
public DelegateAnswer(Log log, Object delegate) {
public DelegateAnswer(org.slf4j.Logger log, Object delegate) {
this.log = log;
this.delegate = delegate;
}

View File

@ -34,6 +34,8 @@
import javax.net.SocketFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -67,7 +69,7 @@
public class TestDFSClientFailover {
private static final Log LOG = LogFactory.getLog(TestDFSClientFailover.class);
private static final Logger LOG = LoggerFactory.getLogger(TestDFSClientFailover.class);
private static final Path TEST_FILE = new Path("/tmp/failover-test-file");
private static final int FILE_LENGTH_TO_VERIFY = 100;
@ -239,7 +241,7 @@ private NameService spyOnNameService() {
List<NameService> nsList = (List<NameService>) f.get(null);
NameService ns = nsList.get(0);
Log log = LogFactory.getLog("NameServiceSpy");
Logger log = LoggerFactory.getLogger("NameServiceSpy");
ns = Mockito.mock(NameService.class,
new GenericTestUtils.DelegateAnswer(log, ns));

View File

@ -32,8 +32,8 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -57,7 +57,7 @@
* using append()/sync() to recover block information
*/
public class TestFileAppend4 {
static final Log LOG = LogFactory.getLog(TestFileAppend4.class);
static final Logger LOG = LoggerFactory.getLogger(TestFileAppend4.class);
static final long BLOCK_SIZE = 1024;
static final long BBW_SIZE = 500; // don't align on bytes/checksum

View File

@ -35,6 +35,8 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FSDataInputStream;
@ -78,8 +80,8 @@ public class TestReplication {
"/d1/r1", "/d1/r1", "/d1/r2", "/d1/r2", "/d1/r2", "/d2/r3", "/d2/r3"
};
private static final int numDatanodes = racks.length;
private static final Log LOG = LogFactory.getLog(
"org.apache.hadoop.hdfs.TestReplication");
private static final Logger LOG = LoggerFactory.getLogger(
TestReplication.class);
/* check if there are at least two nodes are on the same rack */
private void checkFile(FileSystem fileSys, Path name, int repl)

View File

@ -24,8 +24,8 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
@ -43,7 +43,7 @@
import com.google.common.base.Supplier;
public class TestIPCLoggerChannel {
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
TestIPCLoggerChannel.class);
private final Configuration conf = new Configuration();

View File

@ -38,8 +38,8 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
@ -87,7 +87,7 @@
* and messages.
*/
public abstract class BlockReportTestBase {
public static final Log LOG = LogFactory.getLog(BlockReportTestBase.class);
public static final Logger LOG = LoggerFactory.getLogger(BlockReportTestBase.class);
private static short REPL_FACTOR = 1;
private static final int RAND_LIMIT = 2000;
@ -879,7 +879,7 @@ public boolean accept(File file, String s) {
private static void initLoggers() {
DFSTestUtil.setNameNodeLogLevel(Level.ALL);
GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
GenericTestUtils.setLogLevel(BlockReportTestBase.LOG, Level.ALL);
GenericTestUtils.setLogLevel(BlockReportTestBase.LOG, org.slf4j.event.Level.DEBUG);
}
private Block findBlock(Path path, long size) throws IOException {

View File

@ -574,6 +574,19 @@ public static void logStorageContents(Log LOG, NNStorage storage) {
}
}
}
public static void logStorageContents(org.slf4j.Logger LOG, NNStorage storage) {
LOG.info("current storages and corresponding sizes:");
for (StorageDirectory sd : storage.dirIterable(null)) {
File curDir = sd.getCurrentDir();
LOG.info("In directory {}", curDir);
File[] files = curDir.listFiles();
Arrays.sort(files);
for (File f : files) {
LOG.info(" file {}; len = {}", f.getAbsolutePath(), f.length());
}
}
}
/** get the fsImage*/
public static FSImage getFSImage(NameNode node) {

View File

@ -45,6 +45,8 @@
import java.util.List;
import org.apache.commons.cli.ParseException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -110,7 +112,7 @@ public class TestCheckpoint {
GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
}
static final Log LOG = LogFactory.getLog(TestCheckpoint.class);
static final Logger LOG = LoggerFactory.getLogger(TestCheckpoint.class);
static final String NN_METRICS = "NameNodeActivity";
static final long seed = 0xDEADBEEFL;

View File

@ -28,8 +28,8 @@
import java.util.Set;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -74,7 +74,7 @@
*/
public class TestDeleteRace {
private static final int BLOCK_SIZE = 4096;
private static final Log LOG = LogFactory.getLog(TestDeleteRace.class);
private static final Logger LOG = LoggerFactory.getLogger(TestDeleteRace.class);
private static final Configuration conf = new HdfsConfiguration();
private MiniDFSCluster cluster;
@ -171,7 +171,7 @@ public void run() {
inodeMap.put(fileINode);
LOG.info("Deleted" + path);
} catch (Exception e) {
LOG.info(e);
LOG.info(e.toString());
}
}
}
@ -196,7 +196,7 @@ public void run() {
fs.rename(from, to);
LOG.info("Renamed " + from + " to " + to);
} catch (Exception e) {
LOG.info(e);
LOG.info(e.toString());
}
}
}

View File

@ -35,8 +35,8 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -82,7 +82,7 @@ public class TestSaveNamespace {
GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
}
private static final Log LOG = LogFactory.getLog(TestSaveNamespace.class);
private static final Logger LOG = LoggerFactory.getLogger(TestSaveNamespace.class);
private static class FaultySaveImage implements Answer<Void> {
private int count = 0;
@ -325,7 +325,7 @@ public void testReinsertnamedirsInSavenamespace() throws Exception {
try {
fsn.close();
} catch (Throwable t) {
LOG.fatal("Failed to shut down", t);
LOG.error("Failed to shut down", t);
}
}
}
@ -625,7 +625,7 @@ public void testSaveNamespaceWithRenamedLease() throws Exception {
cluster.getNameNodeRpc().saveNamespace(0, 0);
fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
} finally {
IOUtils.cleanup(LOG, out, fs);
IOUtils.cleanupWithLogger(LOG, out, fs);
cluster.shutdown();
}
}

View File

@ -29,8 +29,8 @@
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -69,7 +69,7 @@
public class TestDNFencing {
protected static final Log LOG = LogFactory.getLog(TestDNFencing.class);
protected static final Logger LOG = LoggerFactory.getLogger(TestDNFencing.class);
private static final String TEST_FILE = "/testStandbyIsHot";
private static final Path TEST_FILE_PATH = new Path(TEST_FILE);
private static final int SMALL_BLOCK = 1024;

View File

@ -28,8 +28,8 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -70,12 +70,12 @@
*/
public class TestPipelinesFailover {
static {
GenericTestUtils.setLogLevel(LogFactory.getLog(RetryInvocationHandler
.class), Level.ALL);
GenericTestUtils.setLogLevel(LoggerFactory.getLogger(RetryInvocationHandler
.class), org.slf4j.event.Level.DEBUG);
DFSTestUtil.setNameNodeLogLevel(Level.ALL);
}
protected static final Log LOG = LogFactory.getLog(
protected static final Logger LOG = LoggerFactory.getLogger(
TestPipelinesFailover.class);
private static final Path TEST_PATH =
new Path("/test-file");

View File

@ -21,8 +21,8 @@
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -72,7 +72,7 @@ public class TestStandbyCheckpoints {
private final Random random = new Random();
protected File tmpOivImgDir;
private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
private static final Logger LOG = LoggerFactory.getLogger(TestStandbyCheckpoints.class);
@SuppressWarnings("rawtypes")
@Before