HDFS-9402. Switch DataNode.LOG to use slf4j. Contributed by Walter Su.

This commit is contained in:
Haohui Mai 2015-11-22 15:54:41 -08:00
parent 74d67b2250
commit f504d95e87
11 changed files with 31 additions and 22 deletions

View File

@ -804,6 +804,8 @@ Release 2.8.0 - UNRELEASED
HDFS-9439. Include status of closeAck into exception message in DataNode#run. HDFS-9439. Include status of closeAck into exception message in DataNode#run.
(Xiao Chen via Yongjun Zhang) (Xiao Chen via Yongjun Zhang)
HDFS-9402. Switch DataNode.LOG to use slf4j. (Walter Su via wheat9)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -22,7 +22,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@ -34,6 +33,8 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.protocol.*; import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import org.slf4j.Logger;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
@ -52,8 +53,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
class BPOfferService { class BPOfferService {
static final Log LOG = DataNode.LOG; static final Logger LOG = DataNode.LOG;
/** /**
* Information about the namespace that this service * Information about the namespace that this service
* is registering with. This is assigned after * is registering with. This is assigned after

View File

@ -32,7 +32,6 @@ import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.client.BlockReportOptions; import org.apache.hadoop.hdfs.client.BlockReportOptions;
@ -65,6 +64,7 @@ import org.apache.hadoop.util.VersionUtil;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import org.slf4j.Logger;
/** /**
* A thread per active or standby namenode to perform: * A thread per active or standby namenode to perform:
@ -78,7 +78,7 @@ import com.google.common.collect.Maps;
@InterfaceAudience.Private @InterfaceAudience.Private
class BPServiceActor implements Runnable { class BPServiceActor implements Runnable {
static final Log LOG = DataNode.LOG; static final Logger LOG = DataNode.LOG;
final InetSocketAddress nnAddr; final InetSocketAddress nnAddr;
HAServiceState state; HAServiceState state;
@ -600,7 +600,7 @@ class BPServiceActor implements Runnable {
private synchronized void cleanUp() { private synchronized void cleanUp() {
shouldServiceRun = false; shouldServiceRun = false;
IOUtils.cleanup(LOG, bpNamenode); IOUtils.cleanup(null, bpNamenode);
bpos.shutdownActor(this); bpos.shutdownActor(this);
} }
@ -834,7 +834,7 @@ class BPServiceActor implements Runnable {
sleepAndLogInterrupts(5000, "initializing"); sleepAndLogInterrupts(5000, "initializing");
} else { } else {
runningState = RunningState.FAILED; runningState = RunningState.FAILED;
LOG.fatal("Initialization failed for " + this + ". Exiting. ", ioe); LOG.error("Initialization failed for " + this + ". Exiting. ", ioe);
return; return;
} }
} }

View File

@ -23,7 +23,6 @@ import java.security.PrivilegedExceptionAction;
import java.util.*; import java.util.*;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -35,6 +34,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.slf4j.Logger;
/** /**
* Manages the BPOfferService objects for the data node. * Manages the BPOfferService objects for the data node.
@ -43,7 +43,7 @@ import com.google.common.collect.Sets;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
class BlockPoolManager { class BlockPoolManager {
private static final Log LOG = DataNode.LOG; private static final Logger LOG = DataNode.LOG;
private final Map<String, BPOfferService> bpByNameserviceId = private final Map<String, BPOfferService> bpByNameserviceId =
Maps.newHashMap(); Maps.newHashMap();

View File

@ -63,13 +63,14 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.SYNC_FILE_RANGE_WRITE; import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.SYNC_FILE_RANGE_WRITE;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
/** A class that receives a block and writes to its own disk, meanwhile /** A class that receives a block and writes to its own disk, meanwhile
* may copies it to another site. If a throttler is provided, * may copies it to another site. If a throttler is provided,
* streaming throttling is also supported. * streaming throttling is also supported.
**/ **/
class BlockReceiver implements Closeable { class BlockReceiver implements Closeable {
public static final Log LOG = DataNode.LOG; public static final Logger LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog; static final Log ClientTraceLog = DataNode.ClientTraceLog;
@VisibleForTesting @VisibleForTesting
@ -960,7 +961,7 @@ class BlockReceiver implements Closeable {
// The worst case is not recovering this RBW replica. // The worst case is not recovering this RBW replica.
// Client will fall back to regular pipeline recovery. // Client will fall back to regular pipeline recovery.
} finally { } finally {
IOUtils.cleanup(LOG, out); IOUtils.closeStream(out);
} }
try { try {
// Even if the connection is closed after the ack packet is // Even if the connection is closed after the ack packet is

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -32,6 +31,7 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.slf4j.Logger;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -44,7 +44,7 @@ import java.util.List;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class BlockRecoveryWorker { public class BlockRecoveryWorker {
public static final Log LOG = DataNode.LOG; public static final Logger LOG = DataNode.LOG;
private final DataNode datanode; private final DataNode datanode;
private final Configuration conf; private final Configuration conf;

View File

@ -55,6 +55,7 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_SEQUENTIAL
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.slf4j.Logger;
/** /**
* Reads a block from the disk and sends it to a recipient. * Reads a block from the disk and sends it to a recipient.
@ -97,7 +98,7 @@ import com.google.common.base.Preconditions;
* no checksum error, it replies to DataNode with OP_STATUS_CHECKSUM_OK. * no checksum error, it replies to DataNode with OP_STATUS_CHECKSUM_OK.
*/ */
class BlockSender implements java.io.Closeable { class BlockSender implements java.io.Closeable {
static final Log LOG = DataNode.LOG; static final Logger LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog; static final Log ClientTraceLog = DataNode.ClientTraceLog;
private static final boolean is32Bit = private static final boolean is32Bit =
System.getProperty("sun.arch.data.model").equals("32"); System.getProperty("sun.arch.data.model").equals("32");

View File

@ -212,6 +212,8 @@ import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache; import com.google.common.cache.LoadingCache;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.protobuf.BlockingService; import com.google.protobuf.BlockingService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/********************************************************** /**********************************************************
* DataNode is a class (and program) that stores a set of * DataNode is a class (and program) that stores a set of
@ -248,7 +250,7 @@ import com.google.protobuf.BlockingService;
public class DataNode extends ReconfigurableBase public class DataNode extends ReconfigurableBase
implements InterDatanodeProtocol, ClientDatanodeProtocol, implements InterDatanodeProtocol, ClientDatanodeProtocol,
TraceAdminProtocol, DataNodeMXBean { TraceAdminProtocol, DataNodeMXBean {
public static final Log LOG = LogFactory.getLog(DataNode.class); public static final Logger LOG = LoggerFactory.getLogger(DataNode.class);
static{ static{
HdfsConfiguration.init(); HdfsConfiguration.init();
@ -2602,7 +2604,7 @@ public class DataNode extends ReconfigurableBase
errorCode = 1; errorCode = 1;
} }
} catch (Throwable e) { } catch (Throwable e) {
LOG.fatal("Exception in secureMain", e); LOG.error("Exception in secureMain", e);
terminate(1, e); terminate(1, e);
} finally { } finally {
// We need to terminate the process here because either shutdown was called // We need to terminate the process here because either shutdown was called

View File

@ -89,13 +89,14 @@ import org.apache.hadoop.util.DataChecksum;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
/** /**
* Thread for processing incoming/outgoing data stream. * Thread for processing incoming/outgoing data stream.
*/ */
class DataXceiver extends Receiver implements Runnable { class DataXceiver extends Receiver implements Runnable {
public static final Log LOG = DataNode.LOG; public static final Logger LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog; static final Log ClientTraceLog = DataNode.ClientTraceLog;
private Peer peer; private Peer peer;
@ -376,7 +377,7 @@ class DataXceiver extends Receiver implements Runnable {
blk.getBlockId(), dnR.getDatanodeUuid(), success)); blk.getBlockId(), dnR.getDatanodeUuid(), success));
} }
if (fis != null) { if (fis != null) {
IOUtils.cleanup(LOG, fis); IOUtils.cleanup(null, fis);
} }
} }
} }

View File

@ -22,7 +22,6 @@ import java.net.SocketTimeoutException;
import java.nio.channels.AsynchronousCloseException; import java.nio.channels.AsynchronousCloseException;
import java.util.HashMap; import java.util.HashMap;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.net.Peer;
@ -32,6 +31,7 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
/** /**
* Server used for receiving/sending a block of data. * Server used for receiving/sending a block of data.
@ -40,7 +40,7 @@ import com.google.common.annotations.VisibleForTesting;
* Hadoop IPC mechanism. * Hadoop IPC mechanism.
*/ */
class DataXceiverServer implements Runnable { class DataXceiverServer implements Runnable {
public static final Log LOG = DataNode.LOG; public static final Logger LOG = DataNode.LOG;
private final PeerServer peerServer; private final PeerServer peerServer;
private final DataNode datanode; private final DataNode datanode;
@ -262,7 +262,7 @@ class DataXceiverServer implements Runnable {
synchronized void closeAllPeers() { synchronized void closeAllPeers() {
LOG.info("Closing all peers."); LOG.info("Closing all peers.");
for (Peer p : peers.keySet()) { for (Peer p : peers.keySet()) {
IOUtils.cleanup(LOG, p); IOUtils.cleanup(null, p);
} }
peers.clear(); peers.clear();
peersXceiver.clear(); peersXceiver.clear();

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapred.MiniMRClientClusterFactory; 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.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.log4j.Level; import org.apache.log4j.Level;
@ -45,7 +46,7 @@ public class TestDistCh extends junit.framework.TestCase {
{ {
((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.StateChange") ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.StateChange")
).getLogger().setLevel(Level.ERROR); ).getLogger().setLevel(Level.ERROR);
((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ERROR); GenericTestUtils.setLogLevel(DataNode.LOG, Level.ERROR);
((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ERROR); ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ERROR);
} }