HDFS-13695. Move logging to slf4j in HDFS package. Contributed by Ian Pickering.

This commit is contained in:
Giovanni Matteo Fumarola 2018-09-06 14:48:00 -07:00
parent b3161c4dd9
commit eca1a4bfe9
317 changed files with 1282 additions and 1172 deletions

View File

@ -1872,7 +1872,7 @@ protected Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
return PBHelperClient.convert(
reply.getReadOpChecksumInfo().getChecksum().getType());
} finally {
IOUtilsClient.cleanup(null, pair.in, pair.out);
IOUtilsClient.cleanupWithLogger(LOG, pair.in, pair.out);
}
}
@ -2933,7 +2933,7 @@ public Peer newConnectedPeer(InetSocketAddress addr,
return peer;
} finally {
if (!success) {
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
IOUtils.closeSocket(sock);
}
}

View File

@ -649,7 +649,7 @@ public static Peer peerFromSocketAndKey(
return peer;
} finally {
if (!success) {
IOUtilsClient.cleanup(null, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
}
}
}

View File

@ -188,7 +188,7 @@ public void put(DatanodeID dnId, Peer peer) {
if (peer.isClosed()) return;
if (capacity <= 0) {
// Cache disabled.
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
return;
}
putInternal(dnId, peer);
@ -221,7 +221,7 @@ private synchronized void evictExpired(long expiryPeriod) {
Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
break;
}
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
IOUtilsClient.cleanupWithLogger(LOG, entry.getValue().getPeer());
iter.remove();
}
}
@ -239,7 +239,7 @@ private synchronized void evictOldest() {
"capacity: " + capacity);
}
Entry<Key, Value> entry = iter.next();
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
IOUtilsClient.cleanupWithLogger(LOG, entry.getValue().getPeer());
iter.remove();
}
@ -267,7 +267,7 @@ private void run() throws InterruptedException {
@VisibleForTesting
synchronized void clear() {
for (Value value : multimap.values()) {
IOUtilsClient.cleanup(LOG, value.getPeer());
IOUtilsClient.cleanupWithLogger(LOG, value.getPeer());
}
multimap.clear();
}

View File

@ -549,14 +549,14 @@ public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
// Handle an I/O error we got when using a cached socket.
// These are considered less serious, because the socket may be stale.
LOG.debug("{}: closing stale domain peer {}", this, peer, e);
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
} else {
// Handle an I/O error we got when using a newly created socket.
// We temporarily disable the domain socket path for a few minutes in
// this case, to prevent wasting more time on it.
LOG.warn(this + ": I/O error requesting file descriptors. " +
"Disabling domain socket " + peer.getDomainSocket(), e);
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
clientContext.getDomainSocketFactory()
.disableDomainSocketPath(pathInfo.getPath());
return null;
@ -620,7 +620,7 @@ private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
return null;
} finally {
if (replica == null) {
IOUtilsClient.cleanup(DFSClient.LOG, fis[0], fis[1]);
IOUtilsClient.cleanupWithLogger(DFSClient.LOG, fis[0], fis[1]);
}
}
case ERROR_UNSUPPORTED:
@ -692,7 +692,7 @@ private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
blockReader = getRemoteBlockReader(peer);
return blockReader;
} catch (IOException ioe) {
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
if (isSecurityException(ioe)) {
LOG.trace("{}: got security exception while constructing a remote "
+ " block reader from the unix domain socket at {}",
@ -715,7 +715,7 @@ private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
}
} finally {
if (blockReader == null) {
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
}
}
}
@ -766,7 +766,7 @@ private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
}
} finally {
if (blockReader == null) {
IOUtilsClient.cleanup(LOG, peer);
IOUtilsClient.cleanupWithLogger(LOG, peer);
}
}
}

View File

@ -686,7 +686,7 @@ public synchronized long skip(long n) throws IOException {
@Override
public synchronized void close() throws IOException {
IOUtilsClient.cleanup(LOG, dataIn, checksumIn);
IOUtilsClient.cleanupWithLogger(LOG, dataIn, checksumIn);
if (slowReadBuff != null) {
bufferPool.returnBuffer(slowReadBuff);
slowReadBuff = null;

View File

@ -880,7 +880,7 @@ public void close() {
maxNonMmappedEvictableLifespanMs = 0;
maxEvictableMmapedSize = 0;
// Close and join cacheCleaner thread.
IOUtilsClient.cleanup(LOG, cacheCleaner);
IOUtilsClient.cleanupWithLogger(LOG, cacheCleaner);
// Purge all replicas.
while (true) {
Object eldestKey;
@ -931,7 +931,7 @@ public void close() {
LOG.error("Interrupted while waiting for CleanerThreadPool "
+ "to terminate", e);
}
IOUtilsClient.cleanup(LOG, shmManager);
IOUtilsClient.cleanupWithLogger(LOG, shmManager);
}
@VisibleForTesting // ONLY for testing

View File

@ -244,7 +244,7 @@ void close() {
suffix += " munmapped.";
}
}
IOUtilsClient.cleanup(LOG, dataStream, metaStream);
IOUtilsClient.cleanupWithLogger(LOG, dataStream, metaStream);
if (slot != null) {
cache.scheduleSlotReleaser(slot);
if (LOG.isTraceEnabled()) {

View File

@ -31,7 +31,8 @@ public class IOUtilsClient {
* @param log the log to record problems to at debug level. Can be null.
* @param closeables the objects to close
*/
public static void cleanup(Logger log, java.io.Closeable... closeables) {
public static void cleanupWithLogger(Logger log,
java.io.Closeable... closeables) {
for (java.io.Closeable c : closeables) {
if (c != null) {
try {

View File

@ -1211,11 +1211,11 @@ private void doSingleWrite(final WriteCtx writeCtx) {
LOG.info("Clean up open file context for fileId: {}",
latestAttr.getFileId());
cleanupWithLogger();
cleanup();
}
}
synchronized void cleanupWithLogger() {
synchronized void cleanup() {
if (!activeState) {
LOG.info("Current OpenFileCtx is already inactive, no need to cleanup.");
return;

View File

@ -139,7 +139,7 @@ boolean put(FileHandle h, OpenFileCtx context) {
// Cleanup the old stream outside the lock
if (toEvict != null) {
toEvict.cleanupWithLogger();
toEvict.cleanup();
}
return true;
}
@ -179,7 +179,7 @@ void scan(long streamTimeout) {
// Invoke the cleanup outside the lock
for (OpenFileCtx ofc : ctxToRemove) {
ofc.cleanupWithLogger();
ofc.cleanup();
}
}
@ -215,7 +215,7 @@ void cleanAll() {
// Invoke the cleanup outside the lock
for (OpenFileCtx ofc : cleanedContext) {
ofc.cleanupWithLogger();
ofc.cleanup();
}
}

View File

@ -64,8 +64,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -99,7 +99,8 @@
@InterfaceAudience.Private
public class DFSUtil {
public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(DFSUtil.class.getName());
private DFSUtil() { /* Hidden constructor */ }

View File

@ -21,8 +21,8 @@
import java.io.IOException;
import java.net.URI;
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;
@ -40,7 +40,8 @@
* compilation units. Resolution of fetcher impl will be done at runtime.
*/
public class HdfsDtFetcher implements DtFetcher {
private static final Log LOG = LogFactory.getLog(HdfsDtFetcher.class);
private static final Logger LOG =
LoggerFactory.getLogger(HdfsDtFetcher.class);
private static final String SERVICE_NAME = HdfsConstants.HDFS_URI_SCHEME;

View File

@ -25,8 +25,8 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
@ -73,7 +73,8 @@
@InterfaceAudience.Private
public class NameNodeProxies {
private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
private static final Logger LOG =
LoggerFactory.getLogger(NameNodeProxies.class);
/**
* Creates the namenode proxy with the passed protocol. This will handle

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.web.WebHdfsConstants;
import org.apache.hadoop.io.Text;
@ -28,7 +28,8 @@
* DtFetcher for SWebHdfsFileSystem using the base class HdfsDtFetcher impl.
*/
public class SWebHdfsDtFetcher extends HdfsDtFetcher {
private static final Log LOG = LogFactory.getLog(SWebHdfsDtFetcher.class);
private static final Logger LOG =
LoggerFactory.getLogger(SWebHdfsDtFetcher.class);
private static final String SERVICE_NAME = WebHdfsConstants.SWEBHDFS_SCHEME;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.web.WebHdfsConstants;
import org.apache.hadoop.io.Text;
@ -28,7 +28,8 @@
* DtFetcher for WebHdfsFileSystem using the base class HdfsDtFetcher impl.
*/
public class WebHdfsDtFetcher extends HdfsDtFetcher {
private static final Log LOG = LogFactory.getLog(WebHdfsDtFetcher.class);
private static final Logger LOG =
LoggerFactory.getLogger(WebHdfsDtFetcher.class);
private static final String SERVICE_NAME = WebHdfsConstants.WEBHDFS_SCHEME;

View File

@ -21,14 +21,14 @@
import java.io.IOException;
import java.net.SocketTimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.net.unix.DomainSocket;
@InterfaceAudience.Private
public class DomainPeerServer implements PeerServer {
static final Log LOG = LogFactory.getLog(DomainPeerServer.class);
static final Logger LOG = LoggerFactory.getLogger(DomainPeerServer.class);
private final DomainSocket sock;
DomainPeerServer(DomainSocket sock) {

View File

@ -23,8 +23,8 @@
import java.net.SocketTimeoutException;
import java.nio.channels.ServerSocketChannel;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
@ -32,7 +32,7 @@
@InterfaceAudience.Private
public class TcpPeerServer implements PeerServer {
static final Log LOG = LogFactory.getLog(TcpPeerServer.class);
static final Logger LOG = LoggerFactory.getLogger(TcpPeerServer.class);
private final ServerSocket serverSocket;

View File

@ -23,8 +23,8 @@
import java.util.Map;
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.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -46,7 +46,7 @@
* {@link QuorumCall} instances.
*/
class AsyncLoggerSet {
static final Log LOG = LogFactory.getLog(AsyncLoggerSet.class);
static final Logger LOG = LoggerFactory.getLogger(AsyncLoggerSet.class);
private final List<AsyncLogger> loggers;

View File

@ -30,8 +30,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.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -65,7 +65,7 @@
*/
@InterfaceAudience.Private
public class QuorumJournalManager implements JournalManager {
static final Log LOG = LogFactory.getLog(QuorumJournalManager.class);
static final Logger LOG = LoggerFactory.getLogger(QuorumJournalManager.class);
// Timeouts for which the QJM will wait for each of the following actions.
private final int startSegmentTimeoutMs;

View File

@ -32,8 +32,8 @@
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.text.StringEscapeUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -67,7 +67,8 @@
public class GetJournalEditServlet extends HttpServlet {
private static final long serialVersionUID = -4635891628211723009L;
private static final Log LOG = LogFactory.getLog(GetJournalEditServlet.class);
private static final Logger LOG =
LoggerFactory.getLogger(GetJournalEditServlet.class);
static final String STORAGEINFO_PARAM = "storageInfo";
static final String JOURNAL_ID_PARAM = "jid";

View File

@ -32,8 +32,8 @@
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.Range;
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.FileUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
@ -79,7 +79,7 @@
* the same JVM.
*/
public class Journal implements Closeable {
static final Log LOG = LogFactory.getLog(Journal.class);
static final Logger LOG = LoggerFactory.getLogger(Journal.class);
// Current writing state
@ -1045,7 +1045,7 @@ private void persistPaxosData(long segmentTxId,
public synchronized void doPreUpgrade() throws IOException {
// Do not hold file lock on committedTxnId, because the containing
// directory will be renamed. It will be reopened lazily on next access.
IOUtils.cleanup(LOG, committedTxnId);
IOUtils.cleanupWithLogger(LOG, committedTxnId);
storage.getJournalManager().doPreUpgrade();
}
@ -1087,7 +1087,7 @@ public synchronized void doUpgrade(StorageInfo sInfo) throws IOException {
lastWriterEpoch.set(prevLastWriterEpoch.get());
committedTxnId.set(prevCommittedTxnId.get());
} finally {
IOUtils.cleanup(LOG, prevCommittedTxnId);
IOUtils.cleanupWithLogger(LOG, prevCommittedTxnId);
}
}
@ -1109,7 +1109,7 @@ public Boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
public synchronized void doRollback() throws IOException {
// Do not hold file lock on committedTxnId, because the containing
// directory will be renamed. It will be reopened lazily on next access.
IOUtils.cleanup(LOG, committedTxnId);
IOUtils.cleanupWithLogger(LOG, committedTxnId);
storage.getJournalManager().doRollback();
}

View File

@ -21,8 +21,8 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
@ -68,7 +68,7 @@
*/
@InterfaceAudience.Private
public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
public static final Log LOG = LogFactory.getLog(JournalNode.class);
public static final Logger LOG = LoggerFactory.getLogger(JournalNode.class);
private Configuration conf;
private JournalNodeRpcServer rpcServer;
private JournalNodeHttpServer httpServer;
@ -285,7 +285,7 @@ public void stop(int rc) {
}
for (Journal j : journalsById.values()) {
IOUtils.cleanup(LOG, j);
IOUtils.cleanupWithLogger(LOG, j);
}
DefaultMetricsSystem.shutdown();
@ -403,7 +403,7 @@ private void registerJNMXBean() {
private class ErrorReporter implements StorageErrorReporter {
@Override
public void reportErrorOnFile(File f) {
LOG.fatal("Error reported on file " + f + "... exiting",
LOG.error("Error reported on file " + f + "... exiting",
new Exception());
stop(1);
}

View File

@ -19,7 +19,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingService;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -61,7 +61,7 @@
@VisibleForTesting
public class JournalNodeRpcServer implements QJournalProtocol,
InterQJournalProtocol {
private static final Log LOG = JournalNode.LOG;
private static final Logger LOG = JournalNode.LOG;
private static final int HANDLER_COUNT = 5;
private final JournalNode jn;
private Server server;

View File

@ -29,8 +29,8 @@
import java.util.Map;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -58,7 +58,8 @@
@InterfaceAudience.Private
public class BlockTokenSecretManager extends
SecretManager<BlockTokenIdentifier> {
public static final Log LOG = LogFactory.getLog(BlockTokenSecretManager.class);
public static final Logger LOG =
LoggerFactory.getLogger(BlockTokenSecretManager.class);
public static final Token<BlockTokenIdentifier> DUMMY_TOKEN = new Token<BlockTokenIdentifier>();

View File

@ -27,8 +27,8 @@
import java.util.List;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
@ -62,8 +62,8 @@
public class DelegationTokenSecretManager
extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
private static final Log LOG = LogFactory
.getLog(DelegationTokenSecretManager.class);
private static final Logger LOG = LoggerFactory
.getLogger(DelegationTokenSecretManager.class);
private final FSNamesystem namesystem;
private final SerializerCompat serializerCompat = new SerializerCompat();

View File

@ -36,8 +36,8 @@
import java.util.Set;
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.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -172,7 +172,7 @@
@InterfaceAudience.Private
public class Balancer {
static final Log LOG = LogFactory.getLog(Balancer.class);
static final Logger LOG = LoggerFactory.getLogger(Balancer.class);
static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
@ -724,7 +724,7 @@ static int run(Collection<URI> namenodes, final BalancerParameters p,
}
} finally {
for(NameNodeConnector nnc : connectors) {
IOUtils.cleanup(LOG, nnc);
IOUtils.cleanupWithLogger(LOG, nnc);
}
}
return ExitStatus.SUCCESS.getExitCode();

View File

@ -44,8 +44,8 @@
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
@ -88,7 +88,7 @@
/** Dispatching block replica moves between datanodes. */
@InterfaceAudience.Private
public class Dispatcher {
static final Log LOG = LogFactory.getLog(Dispatcher.class);
static final Logger LOG = LoggerFactory.getLogger(Dispatcher.class);
/**
* the period of time to delay the usage of a DataNode after hitting

View File

@ -31,8 +31,8 @@
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -62,7 +62,8 @@
*/
@InterfaceAudience.Private
public class NameNodeConnector implements Closeable {
private static final Log LOG = LogFactory.getLog(NameNodeConnector.class);
private static final Logger LOG =
LoggerFactory.getLogger(NameNodeConnector.class);
public static final int DEFAULT_MAX_IDLE_ITERATIONS = 5;
private static boolean write2IdFile = true;

View File

@ -25,8 +25,8 @@
import java.util.Random;
import com.google.common.base.Preconditions;
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.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -39,8 +39,8 @@
*/
public class AvailableSpaceBlockPlacementPolicy extends
BlockPlacementPolicyDefault {
private static final Log LOG = LogFactory
.getLog(AvailableSpaceBlockPlacementPolicy.class);
private static final Logger LOG = LoggerFactory
.getLogger(AvailableSpaceBlockPlacementPolicy.class);
private static final Random RAND = new Random();
private int balancedPreference =
(int) (100 * DFS_NAMENODE_AVAILABLE_SPACE_BLOCK_PLACEMENT_POLICY_BALANCED_SPACE_PREFERENCE_FRACTION_DEFAULT);

View File

@ -24,8 +24,8 @@
import com.google.common.base.Preconditions;
import com.google.common.net.InetAddresses;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -73,7 +73,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeManager {
static final Log LOG = LogFactory.getLog(DatanodeManager.class);
static final Logger LOG = LoggerFactory.getLogger(DatanodeManager.class);
private final Namesystem namesystem;
private final BlockManager blockManager;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs.server.blockmanagement;
import com.google.common.annotations.VisibleForTesting;
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.protocol.DatanodeID;
@ -50,7 +50,8 @@
* resolutions are only done during the loading time to minimize the latency.
*/
public class HostFileManager extends HostConfigManager {
private static final Log LOG = LogFactory.getLog(HostFileManager.class);
private static final Logger LOG =
LoggerFactory.getLogger(HostFileManager.class);
private Configuration conf;
private HostSet includes = new HostSet();
private HostSet excludes = new HostSet();

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs.server.common;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@ -53,7 +53,7 @@ public class JspHelper {
public static final String CURRENT_CONF = "current.conf";
public static final String DELEGATION_PARAMETER_NAME = DelegationParam.NAME;
public static final String NAMENODE_ADDRESS = "nnaddr";
private static final Log LOG = LogFactory.getLog(JspHelper.class);
private static final Logger LOG = LoggerFactory.getLogger(JspHelper.class);
/** Private constructor for preventing creating JspHelper object. */
private JspHelper() {}

View File

@ -32,8 +32,9 @@
import javax.management.ObjectName;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
@ -43,7 +44,8 @@
*/
public class MetricsLoggerTask implements Runnable {
public static final Log LOG = LogFactory.getLog(MetricsLoggerTask.class);
public static final Logger LOG =
LoggerFactory.getLogger(MetricsLoggerTask.class);
private static ObjectName objectName = null;

View File

@ -38,8 +38,8 @@
import com.google.common.base.Preconditions;
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.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -57,7 +57,8 @@
@InterfaceAudience.Private
public final class Util {
private final static Log LOG = LogFactory.getLog(Util.class.getName());
private final static Logger LOG =
LoggerFactory.getLogger(Util.class.getName());
public final static String FILE_LENGTH = "File-Length";
public final static String CONTENT_LENGTH = "Content-Length";

View File

@ -39,8 +39,8 @@
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.AutoCloseableLock;
@ -60,7 +60,8 @@
*/
@InterfaceAudience.Private
public class DirectoryScanner implements Runnable {
private static final Log LOG = LogFactory.getLog(DirectoryScanner.class);
private static final Logger LOG =
LoggerFactory.getLogger(DirectoryScanner.class);
private static final int MILLIS_PER_SECOND = 1000;
private static final String START_MESSAGE =
"Periodic Directory Tree Verification scan"

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -37,7 +37,8 @@
*/
@InterfaceAudience.Private
class ProfilingFileIoEvents {
static final Log LOG = LogFactory.getLog(ProfilingFileIoEvents.class);
static final Logger LOG =
LoggerFactory.getLogger(ProfilingFileIoEvents.class);
private final boolean isEnabled;
private final int sampleRangeMax;

View File

@ -32,8 +32,8 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.io.IOUtils;
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.InvalidRequestException;
import org.apache.hadoop.hdfs.ExtendedBlockId;
@ -81,7 +81,8 @@
* The counterpart of this class on the client is {@link DfsClientShmManager}.
*/
public class ShortCircuitRegistry {
public static final Log LOG = LogFactory.getLog(ShortCircuitRegistry.class);
public static final Logger LOG =
LoggerFactory.getLogger(ShortCircuitRegistry.class);
private static final int SHM_LENGTH = 8192;

View File

@ -27,8 +27,8 @@
import java.util.List;
import java.util.Random;
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.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
@ -46,7 +46,8 @@
public class AvailableSpaceVolumeChoosingPolicy<V extends FsVolumeSpi>
implements VolumeChoosingPolicy<V>, Configurable {
private static final Log LOG = LogFactory.getLog(AvailableSpaceVolumeChoosingPolicy.class);
private static final Logger LOG =
LoggerFactory.getLogger(AvailableSpaceVolumeChoosingPolicy.class);
private Object[] syncLocks;

View File

@ -20,8 +20,8 @@
import java.io.IOException;
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.hadoop.fs.StorageType;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
@ -31,7 +31,8 @@
*/
public class RoundRobinVolumeChoosingPolicy<V extends FsVolumeSpi>
implements VolumeChoosingPolicy<V> {
public static final Log LOG = LogFactory.getLog(RoundRobinVolumeChoosingPolicy.class);
public static final Logger LOG =
LoggerFactory.getLogger(RoundRobinVolumeChoosingPolicy.class);
// curVolumes stores the RR counters of each storage type.
// The ordinal of storage type in org.apache.hadoop.fs.StorageType

View File

@ -32,8 +32,8 @@
import java.util.Scanner;
import java.util.concurrent.atomic.AtomicLong;
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.CachingGetSpaceUsed;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -71,7 +71,7 @@
* This class is synchronized by {@link FsVolumeImpl}.
*/
class BlockPoolSlice {
static final Log LOG = LogFactory.getLog(BlockPoolSlice.class);
static final Logger LOG = LoggerFactory.getLogger(BlockPoolSlice.class);
private final String bpid;
private final FsVolumeImpl volume; // volume to which this BlockPool belongs to
@ -764,7 +764,7 @@ void shutdown(BlockListAsLongs blocksListToPersist) {
}
if (dfsUsage instanceof CachingGetSpaceUsed) {
IOUtils.cleanup(LOG, ((CachingGetSpaceUsed) dfsUsage));
IOUtils.cleanupWithLogger(LOG, ((CachingGetSpaceUsed) dfsUsage));
}
}

View File

@ -30,8 +30,8 @@
import java.util.concurrent.ThreadPoolExecutor;
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.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@ -59,7 +59,8 @@
* They should be combined.
*/
class FsDatasetAsyncDiskService {
public static final Log LOG = LogFactory.getLog(FsDatasetAsyncDiskService.class);
public static final Logger LOG =
LoggerFactory.getLogger(FsDatasetAsyncDiskService.class);
// ThreadPool core pool size
private static final int CORE_THREADS_PER_VOLUME = 1;

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
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.DFSUtilClient;
import org.apache.hadoop.hdfs.HdfsConfiguration;
@ -45,7 +45,8 @@
* They should be combined.
*/
class RamDiskAsyncLazyPersistService {
public static final Log LOG = LogFactory.getLog(RamDiskAsyncLazyPersistService.class);
public static final Logger LOG =
LoggerFactory.getLogger(RamDiskAsyncLazyPersistService.class);
// ThreadPool core pool size
private static final int CORE_THREADS_PER_VOLUME = 1;

View File

@ -20,8 +20,8 @@
import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -36,7 +36,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public abstract class RamDiskReplicaTracker {
static final Log LOG = LogFactory.getLog(RamDiskReplicaTracker.class);
static final Logger LOG =
LoggerFactory.getLogger(RamDiskReplicaTracker.class);
FsDatasetImpl fsDataset;

View File

@ -41,8 +41,8 @@
import io.netty.handler.ssl.SslHandler;
import io.netty.handler.stream.ChunkedWriteHandler;
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.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -87,7 +87,7 @@ public class DatanodeHttpServer implements Closeable {
private final RestCsrfPreventionFilter restCsrfPreventionFilter;
private InetSocketAddress httpAddress;
private InetSocketAddress httpsAddress;
static final Log LOG = LogFactory.getLog(DatanodeHttpServer.class);
static final Logger LOG = LoggerFactory.getLogger(DatanodeHttpServer.class);
// HttpServer threads are only used for the web UI and basic servlets, so
// set them to the minimum possible

View File

@ -30,7 +30,7 @@
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.util.ReferenceCountUtil;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
@ -46,7 +46,7 @@
final class RestCsrfPreventionFilterHandler
extends SimpleChannelInboundHandler<HttpRequest> {
private static final Log LOG = DatanodeHttpServer.LOG;
private static final Logger LOG = DatanodeHttpServer.LOG;
private final RestCsrfPreventionFilter restCsrfPreventionFilter;

View File

@ -34,7 +34,7 @@
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpRequestEncoder;
import io.netty.handler.codec.http.HttpResponseEncoder;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import java.net.InetSocketAddress;
@ -53,7 +53,7 @@ class SimpleHttpProxyHandler extends SimpleChannelInboundHandler<HttpRequest> {
private String uri;
private Channel proxiedChannel;
private final InetSocketAddress host;
static final Log LOG = DatanodeHttpServer.LOG;
static final Logger LOG = DatanodeHttpServer.LOG;
SimpleHttpProxyHandler(InetSocketAddress host) {
this.host = host;

View File

@ -13,8 +13,8 @@
*/
package org.apache.hadoop.hdfs.server.datanode.web.webhdfs;
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.security.token.delegation.DelegationTokenIdentifier;
@ -43,7 +43,7 @@ public class DataNodeUGIProvider {
private final ParameterParser params;
@VisibleForTesting
static Cache<String, UserGroupInformation> ugiCache;
public static final Log LOG = LogFactory.getLog(Client.class);
public static final Logger LOG = LoggerFactory.getLogger(Client.class);
DataNodeUGIProvider(ParameterParser params) {
this.params = params;

View File

@ -23,7 +23,7 @@
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
@ -43,7 +43,7 @@
import static org.apache.hadoop.hdfs.server.datanode.web.webhdfs.WebHdfsHandler.APPLICATION_JSON_UTF8;
class ExceptionHandler {
static Log LOG = WebHdfsHandler.LOG;
private static final Logger LOG = WebHdfsHandler.LOG;
static DefaultFullHttpResponse exceptionCaught(Throwable cause) {
Exception e = cause instanceof Exception ? (Exception) cause : new Exception(cause);

View File

@ -23,7 +23,7 @@
import io.netty.handler.codec.http.DefaultHttpResponse;
import io.netty.handler.codec.http.HttpContent;
import io.netty.handler.codec.http.LastHttpContent;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.io.IOUtils;
@ -37,7 +37,7 @@ class HdfsWriter extends SimpleChannelInboundHandler<HttpContent> {
private final DFSClient client;
private final OutputStream out;
private final DefaultHttpResponse response;
private static final Log LOG = WebHdfsHandler.LOG;
private static final Logger LOG = WebHdfsHandler.LOG;
HdfsWriter(DFSClient client, OutputStream out, DefaultHttpResponse response) {
this.client = client;
@ -82,8 +82,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
}
private void releaseDfsResources() {
IOUtils.cleanup(LOG, out);
IOUtils.cleanup(LOG, client);
IOUtils.cleanupWithLogger(LOG, out);
IOUtils.cleanupWithLogger(LOG, client);
}
private void releaseDfsResourcesAndThrow() throws Exception {

View File

@ -29,8 +29,8 @@
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.QueryStringDecoder;
import io.netty.handler.stream.ChunkedStream;
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.CreateFlag;
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
@ -87,8 +87,8 @@
import static org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier.HDFS_DELEGATION_KIND;
public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
static final Log LOG = LogFactory.getLog(WebHdfsHandler.class);
static final Log REQLOG = LogFactory.getLog("datanode.webhdfs");
static final Logger LOG = LoggerFactory.getLogger(WebHdfsHandler.class);
static final Logger REQLOG = LoggerFactory.getLogger("datanode.webhdfs");
public static final String WEBHDFS_PREFIX = WebHdfsFileSystem.PATH_PREFIX;
public static final int WEBHDFS_PREFIX_LENGTH = WEBHDFS_PREFIX.length();
public static final String APPLICATION_OCTET_STREAM =
@ -295,7 +295,7 @@ private void onGetFileChecksum(ChannelHandlerContext ctx) throws IOException {
dfsclient.close();
dfsclient = null;
} finally {
IOUtils.cleanup(LOG, dfsclient);
IOUtils.cleanupWithLogger(LOG, dfsclient);
}
final byte[] js =
JsonUtil.toJsonString(checksum).getBytes(StandardCharsets.UTF_8);

View File

@ -22,8 +22,8 @@
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.commons.cli.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
@ -70,7 +70,7 @@
@InterfaceAudience.Private
public class Mover {
static final Log LOG = LogFactory.getLog(Mover.class);
static final Logger LOG = LoggerFactory.getLogger(Mover.class);
private static class StorageMap {
private final StorageGroupMap<Source> sources
@ -659,7 +659,7 @@ static int run(Map<URI, List<Path>> namenodes, Configuration conf)
final ExitStatus r = m.run();
if (r == ExitStatus.SUCCESS) {
IOUtils.cleanup(LOG, nnc);
IOUtils.cleanupWithLogger(LOG, nnc);
iter.remove();
} else if (r != ExitStatus.IN_PROGRESS) {
if (r == ExitStatus.NO_MOVE_PROGRESS) {
@ -682,7 +682,7 @@ static int run(Map<URI, List<Path>> namenodes, Configuration conf)
return ExitStatus.SUCCESS.getExitCode();
} finally {
for (NameNodeConnector nnc : connectors) {
IOUtils.cleanup(LOG, nnc);
IOUtils.cleanupWithLogger(LOG, nnc);
}
}
}
@ -720,7 +720,7 @@ private static String[] readPathFile(String file) throws IOException {
}
}
} finally {
IOUtils.cleanup(LOG, reader);
IOUtils.cleanupWithLogger(LOG, reader);
}
return list.toArray(new String[list.size()]);
}

View File

@ -21,8 +21,6 @@
import javax.annotation.Nonnull;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;

View File

@ -19,8 +19,8 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -30,7 +30,8 @@
@InterfaceAudience.Private
public class CheckpointConf {
private static final Log LOG = LogFactory.getLog(CheckpointConf.class);
private static final Logger LOG =
LoggerFactory.getLogger(CheckpointConf.class);
/** How often to checkpoint regardless of number of txns */
private final long checkpointPeriod; // in seconds

View File

@ -28,8 +28,8 @@
import java.net.URL;
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.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@ -56,8 +56,8 @@
* (1) time or (2) the size of the edits file.
*/
class Checkpointer extends Daemon {
public static final Log LOG =
LogFactory.getLog(Checkpointer.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(Checkpointer.class.getName());
private final BackupNode backupNode;
volatile boolean shouldRun;

View File

@ -22,8 +22,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.security.AccessControlException;
@ -48,8 +48,8 @@ public class ContentSummaryComputationContext {
private int sleepNanoSec = 0;
public static final String REPLICATED = "Replicated";
public static final Log LOG = LogFactory
.getLog(ContentSummaryComputationContext.class);
public static final Logger LOG = LoggerFactory
.getLogger(ContentSummaryComputationContext.class);
private FSPermissionChecker pc;
/**

View File

@ -21,8 +21,8 @@
import javax.servlet.http.HttpServletRequest;
import java.io.IOException;
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.server.common.JspHelper;
import org.apache.hadoop.security.UserGroupInformation;
@ -34,7 +34,8 @@ abstract class DfsServlet extends HttpServlet {
/** For java.io.Serializable */
private static final long serialVersionUID = 1L;
static final Log LOG = LogFactory.getLog(DfsServlet.class.getCanonicalName());
static final Logger LOG =
LoggerFactory.getLogger(DfsServlet.class.getCanonicalName());
protected UserGroupInformation getUGI(HttpServletRequest request,
Configuration conf) throws IOException {

View File

@ -21,8 +21,8 @@
import java.net.InetSocketAddress;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.server.common.Storage;
@ -43,7 +43,8 @@
* int, int, byte[])
*/
class EditLogBackupOutputStream extends EditLogOutputStream {
private static final Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
private static final Logger LOG =
LoggerFactory.getLogger(EditLogFileOutputStream.class);
static final int DEFAULT_BUFFER_SIZE = 256;
private final JournalProtocol backupNode; // RPC proxy to backup node

View File

@ -29,8 +29,8 @@
import java.net.URL;
import java.security.PrivilegedExceptionAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.LayoutFlags;
@ -71,7 +71,7 @@ static private enum State {
private FSEditLogOp.Reader reader = null;
private FSEditLogLoader.PositionTrackingInputStream tracker = null;
private DataInputStream dataIn = null;
static final Log LOG = LogFactory.getLog(EditLogInputStream.class);
static final Logger LOG = LoggerFactory.getLogger(EditLogInputStream.class);
/**
* Open an EditLogInputStream for the given file.
@ -161,7 +161,7 @@ private void init(boolean verifyLayoutVersion)
state = State.OPEN;
} finally {
if (reader == null) {
IOUtils.cleanup(LOG, dataIn, tracker, bin, fStream);
IOUtils.cleanupWithLogger(LOG, dataIn, tracker, bin, fStream);
state = State.CLOSED;
}
}

View File

@ -26,8 +26,8 @@
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -42,7 +42,8 @@
*/
@InterfaceAudience.Private
public class EditLogFileOutputStream extends EditLogOutputStream {
private static final Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
private static final Logger LOG =
LoggerFactory.getLogger(EditLogFileOutputStream.class);
public static final int MIN_PREALLOCATION_LENGTH = 1024 * 1024;
private File file;
@ -161,7 +162,7 @@ public void close() throws IOException {
fp.close();
fp = null;
} finally {
IOUtils.cleanup(LOG, fc, fp);
IOUtils.cleanupWithLogger(LOG, fc, fp);
doubleBuf = null;
fc = null;
fp = null;
@ -174,7 +175,7 @@ public void abort() throws IOException {
if (fp == null) {
return;
}
IOUtils.cleanup(LOG, fp);
IOUtils.cleanupWithLogger(LOG, fp);
fp = null;
}

View File

@ -24,8 +24,8 @@
import java.util.Arrays;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
@ -43,7 +43,8 @@
*/
@InterfaceAudience.Private
public class EditsDoubleBuffer {
protected static final Log LOG = LogFactory.getLog(EditsDoubleBuffer.class);
protected static final Logger LOG =
LoggerFactory.getLogger(EditsDoubleBuffer.class);
private TxnBuffer bufCurrent; // current buffer for writing
private TxnBuffer bufReady; // buffer ready for flushing

View File

@ -26,8 +26,8 @@
import java.util.concurrent.BlockingQueue;
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.ipc.Server;
import org.apache.hadoop.util.ExitUtil;
@ -35,7 +35,7 @@
import com.google.common.base.Preconditions;
class FSEditLogAsync extends FSEditLog implements Runnable {
static final Log LOG = LogFactory.getLog(FSEditLog.class);
static final Logger LOG = LoggerFactory.getLogger(FSEditLog.class);
// use separate mutex to avoid possible deadlock when stopping the thread.
private final Object syncThreadLock = new Object();
@ -203,7 +203,7 @@ public void run() {
private void terminate(Throwable t) {
String message = "Exception while edit logging: "+t.getMessage();
LOG.fatal(message, t);
LOG.error(message, t);
ExitUtil.terminate(1, message);
}

View File

@ -28,8 +28,8 @@
import java.util.EnumSet;
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.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
@ -121,7 +121,8 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FSEditLogLoader {
static final Log LOG = LogFactory.getLog(FSEditLogLoader.class.getName());
static final Logger LOG =
LoggerFactory.getLogger(FSEditLogLoader.class.getName());
static final long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec
private final FSNamesystem fsNamesys;

View File

@ -37,8 +37,8 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -83,7 +83,8 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FSImage implements Closeable {
public static final Log LOG = LogFactory.getLog(FSImage.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(FSImage.class.getName());
protected FSEditLog editLog = null;
private boolean isUpgradeFinalized = false;
@ -1136,7 +1137,7 @@ public synchronized void saveNamespace(FSNamesystem source, NameNodeFile nnf,
getStorage().updateNameDirSize();
if (exitAfterSave.get()) {
LOG.fatal("NameNode process will exit now... The saved FsImage " +
LOG.error("NameNode process will exit now... The saved FsImage " +
nnf + " is potentially corrupted.");
ExitUtil.terminate(-1);
}

View File

@ -37,7 +37,7 @@
import java.util.Map;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -181,7 +181,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FSImageFormat {
private static final Log LOG = FSImage.LOG;
private static final Logger LOG = FSImage.LOG;
// Static-only class
private FSImageFormat() {}
@ -231,7 +231,7 @@ public void load(File file, boolean requireSameLayoutVersion)
loader.load(file);
}
} finally {
IOUtils.cleanup(LOG, is);
IOUtils.cleanupWithLogger(LOG, is);
}
}
}

View File

@ -26,8 +26,8 @@
import java.util.Iterator;
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.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.AclEntry;
@ -104,7 +104,8 @@ public final class FSImageFormatPBINode {
XAttr.NameSpace.values();
private static final Log LOG = LogFactory.getLog(FSImageFormatPBINode.class);
private static final Logger LOG =
LoggerFactory.getLogger(FSImageFormatPBINode.class);
public final static class Loader {
public static PermissionStatus loadPermission(long id,

View File

@ -29,8 +29,8 @@
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileUtil;
@ -51,8 +51,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
private static final Log LOG =
LogFactory.getLog(FSImagePreTransactionalStorageInspector.class);
private static final Logger LOG =
LoggerFactory.getLogger(FSImagePreTransactionalStorageInspector.class);
/* Flag if there is at least one storage dir that doesn't contain the newest
* fstime */
@ -136,7 +136,7 @@ static long readCheckpointTime(StorageDirectory sd) throws IOException {
in.close();
in = null;
} finally {
IOUtils.cleanup(LOG, in);
IOUtils.cleanupWithLogger(LOG, in);
}
}
return timeStamp;

View File

@ -28,8 +28,8 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -39,7 +39,7 @@
import com.google.common.collect.Lists;
class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
public static final Log LOG = LogFactory.getLog(
public static final Logger LOG = LoggerFactory.getLogger(
FSImageTransactionalStorageInspector.class);
private boolean needToSave = false;

View File

@ -307,9 +307,9 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.apache.log4j.Logger;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
import org.apache.log4j.Logger;
import org.eclipse.jetty.util.ajax.JSON;
import com.google.common.annotations.VisibleForTesting;

View File

@ -24,8 +24,8 @@
import java.util.Stack;
import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.permission.AclEntryScope;
@ -47,7 +47,7 @@
* Some of the helper methods are guarded by {@link FSNamesystem#readLock()}.
*/
public class FSPermissionChecker implements AccessControlEnforcer {
static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
static final Logger LOG = LoggerFactory.getLogger(UserGroupInformation.class);
private static String getPath(byte[][] components, int start, int end) {
return DFSUtil.byteArray2PathString(components, start, end - start + 1);

View File

@ -17,8 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
@ -60,7 +60,8 @@
*/
@InterfaceAudience.Private
public class FileJournalManager implements JournalManager {
private static final Log LOG = LogFactory.getLog(FileJournalManager.class);
private static final Logger LOG =
LoggerFactory.getLogger(FileJournalManager.class);
private final Configuration conf;
private final StorageDirectory sd;

View File

@ -25,8 +25,8 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Path;
@ -56,7 +56,7 @@
*/
@InterfaceAudience.Private
public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
public static final Log LOG = LogFactory.getLog(INode.class);
public static final Logger LOG = LoggerFactory.getLogger(INode.class);
/** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
private INode parent = null;

View File

@ -19,8 +19,8 @@
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -36,7 +36,7 @@
* Contains INodes information resolved from a given path.
*/
public class INodesInPath {
public static final Log LOG = LogFactory.getLog(INodesInPath.class);
public static final Logger LOG = LoggerFactory.getLogger(INodesInPath.class);
/**
* @return true if path component is {@link HdfsConstants#DOT_SNAPSHOT_DIR}

View File

@ -35,8 +35,8 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
@ -72,7 +72,7 @@ public class ImageServlet extends HttpServlet {
private static final long serialVersionUID = -7669068179452648952L;
private static final Log LOG = LogFactory.getLog(ImageServlet.class);
private static final Logger LOG = LoggerFactory.getLogger(ImageServlet.class);
public final static String CONTENT_DISPOSITION = "Content-Disposition";
public final static String HADOOP_IMAGE_EDITS_HEADER = "X-Image-Edits-Name";

View File

@ -30,8 +30,8 @@
import java.util.SortedSet;
import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
@ -54,7 +54,7 @@
@InterfaceAudience.Private
public class JournalSet implements JournalManager {
static final Log LOG = LogFactory.getLog(FSEditLog.class);
static final Logger LOG = LoggerFactory.getLogger(FSEditLog.class);
// we want local logs to be ordered earlier in the collection, and true
// is considered larger than false, so reverse the comparator
@ -387,7 +387,7 @@ private void mapJournalsAndReportErrors(
if (jas.isRequired()) {
final String msg = "Error: " + status + " failed for required journal ("
+ jas + ")";
LOG.fatal(msg, t);
LOG.error(msg, t);
// If we fail on *any* of the required journals, then we must not
// continue on any of the other journals. Abort them to ensure that
// retry behavior doesn't allow them to keep going in any way.

View File

@ -22,14 +22,15 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Context data for an ongoing NameNode metadata recovery process. */
@InterfaceAudience.Private
@InterfaceStability.Evolving
public final class MetaRecoveryContext {
public static final Log LOG = LogFactory.getLog(MetaRecoveryContext.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(MetaRecoveryContext.class.getName());
public final static int FORCE_NONE = 0;
public final static int FORCE_FIRST_CHOICE = 1;
public final static int FORCE_ALL = 2;

View File

@ -28,8 +28,8 @@
import java.util.List;
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.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
@ -56,7 +56,7 @@ public class NNStorageRetentionManager {
private final int numCheckpointsToRetain;
private final long numExtraEditsToRetain;
private final int maxExtraEditsSegmentsToRetain;
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
NNStorageRetentionManager.class);
private final NNStorage storage;
private final StoragePurger purger;

View File

@ -27,8 +27,8 @@
import java.nio.file.attribute.BasicFileAttributes;
import java.util.Collections;
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.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@ -38,7 +38,8 @@
public abstract class NNUpgradeUtil {
private static final Log LOG = LogFactory.getLog(NNUpgradeUtil.class);
private static final Logger LOG =
LoggerFactory.getLogger(NNUpgradeUtil.class);
/**
* Return true if this storage dir can roll back to the previous storage

View File

@ -20,8 +20,8 @@
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Caches frequently used names to facilitate reuse.
@ -62,7 +62,7 @@ int get() {
}
}
static final Log LOG = LogFactory.getLog(NameCache.class.getName());
static final Logger LOG = LoggerFactory.getLogger(NameCache.class.getName());
/** indicates initialization is in progress */
private boolean initialized = false;

View File

@ -25,8 +25,8 @@
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DF;
@ -47,7 +47,8 @@
*/
@InterfaceAudience.Private
public class NameNodeResourceChecker {
private static final Log LOG = LogFactory.getLog(NameNodeResourceChecker.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(NameNodeResourceChecker.class.getName());
// Space (in bytes) reserved per volume.
private final long duReserved;

View File

@ -36,8 +36,8 @@
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@ -112,7 +112,8 @@
*/
@InterfaceAudience.Private
public class NamenodeFsck implements DataEncryptionKeyFactory {
public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(NameNode.class.getName());
// return string marking fsck status
public static final String CORRUPT_STATUS = "is CORRUPT";
@ -358,7 +359,7 @@ public void fsck() {
blockIdCK(blk);
sb.append(blk + "\n");
}
LOG.info(sb);
LOG.info("{}", sb.toString());
namenode.getNamesystem().logFsckEvent("/", remoteAddress);
out.flush();
return;

View File

@ -21,8 +21,8 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.io.IOUtils;
@ -37,7 +37,7 @@
* different subset of the available edits.
*/
class RedundantEditLogInputStream extends EditLogInputStream {
public static final Log LOG = LogFactory.getLog(
public static final Logger LOG = LoggerFactory.getLogger(
RedundantEditLogInputStream.class.getName());
private int curIdx;
private long prevTxId;
@ -152,7 +152,7 @@ public long getLastTxId() {
@Override
public void close() throws IOException {
IOUtils.cleanup(LOG, streams);
IOUtils.cleanupWithLogger(LOG, streams);
}
@Override

View File

@ -38,8 +38,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -103,8 +103,8 @@ public class SecondaryNameNode implements Runnable,
static{
HdfsConfiguration.init();
}
public static final Log LOG =
LogFactory.getLog(SecondaryNameNode.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(SecondaryNameNode.class.getName());
private final long starttime = Time.now();
private volatile long lastCheckpointTime = 0;
@ -367,12 +367,12 @@ public void doWork() {
// Prevent a huge number of edits from being created due to
// unrecoverable conditions and endless retries.
if (checkpointImage.getMergeErrorCount() > maxRetries) {
LOG.fatal("Merging failed " +
LOG.error("Merging failed " +
checkpointImage.getMergeErrorCount() + " times.");
terminate(1);
}
} catch (Throwable e) {
LOG.fatal("Throwable Exception in doCheckpoint", e);
LOG.error("Throwable Exception in doCheckpoint", e);
e.printStackTrace();
terminate(1, e);
}
@ -676,7 +676,7 @@ boolean shouldCheckpointBasedOnCount() throws IOException {
public static void main(String[] argv) throws Exception {
CommandLineOpts opts = SecondaryNameNode.parseArgs(argv);
if (opts == null) {
LOG.fatal("Failed to parse options");
LOG.error("Failed to parse options");
terminate(1);
} else if (opts.shouldPrintHelp()) {
opts.usage();
@ -703,7 +703,7 @@ public static void main(String[] argv) throws Exception {
secondary.join();
}
} catch (Throwable e) {
LOG.fatal("Failed to start secondary namenode", e);
LOG.error("Failed to start secondary namenode", e);
terminate(1);
}
}

View File

@ -103,7 +103,7 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp)
json.writeEndArray();
json.writeEndObject();
} finally {
IOUtils.cleanup(LOG, json);
IOUtils.cleanupWithLogger(LOG, json);
}
}

View File

@ -32,8 +32,8 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
@ -95,7 +95,8 @@ public static TransferResult getResultForCode(int code){
@VisibleForTesting
static int timeout = 0;
private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
private static final Logger LOG =
LoggerFactory.getLogger(TransferFsImage.class);
public static void downloadMostRecentImageToDirectory(URL infoServer,
File dir) throws IOException {

View File

@ -31,8 +31,8 @@
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
@ -76,7 +76,8 @@
*/
@InterfaceAudience.Private
public class BootstrapStandby implements Tool, Configurable {
private static final Log LOG = LogFactory.getLog(BootstrapStandby.class);
private static final Logger LOG =
LoggerFactory.getLogger(BootstrapStandby.class);
private String nsId;
private String nnId;
private List<RemoteNameNodeInfo> remoteNNs;
@ -182,14 +183,14 @@ private int doRun() throws IOException {
}
if (nsInfo == null) {
LOG.fatal(
LOG.error(
"Unable to fetch namespace information from any remote NN. Possible NameNodes: "
+ remoteNNs);
return ERR_CODE_FAILED_CONNECT;
}
if (!checkLayoutVersion(nsInfo)) {
LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
LOG.error("Layout version on remote node (" + nsInfo.getLayoutVersion()
+ ") does not match " + "this node's layout version ("
+ HdfsServerConstants.NAMENODE_LAYOUT_VERSION + ")");
return ERR_CODE_INVALID_VERSION;
@ -382,7 +383,7 @@ private boolean checkLogsAvailableForRead(FSImage image, long imageTxId,
"Please copy these logs into the shared edits storage " +
"or call saveNamespace on the active node.\n" +
"Error: " + e.getLocalizedMessage();
LOG.fatal(msg, e);
LOG.error(msg, e);
return false;
}

View File

@ -36,8 +36,8 @@
import com.google.common.collect.Iterators;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -73,7 +73,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class EditLogTailer {
public static final Log LOG = LogFactory.getLog(EditLogTailer.class);
public static final Logger LOG = LoggerFactory.getLogger(EditLogTailer.class);
/**
* StandbyNode will hold namesystem lock to apply at most this many journal
@ -471,7 +471,7 @@ private void doWork() {
// interrupter should have already set shouldRun to false
continue;
} catch (Throwable t) {
LOG.fatal("Unknown error encountered while tailing edits. " +
LOG.error("Unknown error encountered while tailing edits. " +
"Shutting down standby NN.", t);
terminate(1, t);
}

View File

@ -56,8 +56,8 @@
import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.Response.Status;
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.ContentSummary;
import org.apache.hadoop.fs.FileEncryptionInfo;
@ -117,7 +117,8 @@
@Path("")
@ResourceFilters(ParamFilter.class)
public class NamenodeWebHdfsMethods {
public static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
public static final Logger LOG =
LoggerFactory.getLogger(NamenodeWebHdfsMethods.class);
private static final UriFsPathParam ROOT = new UriFsPathParam("");

View File

@ -40,8 +40,8 @@
import com.google.common.base.Joiner;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.ReconfigurationTaskStatus;
@ -114,7 +114,7 @@ public class DFSAdmin extends FsShell {
HdfsConfiguration.init();
}
private static final Log LOG = LogFactory.getLog(DFSAdmin.class);
private static final Logger LOG = LoggerFactory.getLogger(DFSAdmin.class);
/**
* An abstract class for the execution of a file system command

View File

@ -21,8 +21,8 @@
import java.util.Arrays;
import java.util.Collection;
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.CommonConfigurationKeys;
import org.apache.hadoop.ha.HAAdmin;
@ -38,7 +38,7 @@
*/
public class DFSHAAdmin extends HAAdmin {
private static final Log LOG = LogFactory.getLog(DFSHAAdmin.class);
private static final Logger LOG = LoggerFactory.getLogger(DFSHAAdmin.class);
private String nameserviceId;

View File

@ -30,8 +30,8 @@
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -61,8 +61,8 @@
@InterfaceAudience.Private
public class DFSZKFailoverController extends ZKFailoverController {
private static final Log LOG =
LogFactory.getLog(DFSZKFailoverController.class);
private static final Logger LOG =
LoggerFactory.getLogger(DFSZKFailoverController.class);
private final AccessControlList adminAcl;
/* the same as superclass's localTarget, but with the more specfic NN type */
private final NNHAServiceTarget localNNTarget;
@ -194,7 +194,7 @@ public static void main(String args[])
try {
System.exit(zkfc.run(parser.getRemainingArgs()));
} catch (Throwable t) {
LOG.fatal("DFSZKFailOverController exiting due to earlier exception "
LOG.error("DFSZKFailOverController exiting due to earlier exception "
+ t);
terminate(1, t);
}
@ -243,7 +243,7 @@ private void getLocalNNThreadDump() {
new StringBuilder("-- Local NN thread dump -- \n");
localNNThreadDumpContent.append(out);
localNNThreadDumpContent.append("\n -- Local NN thread dump -- ");
LOG.info(localNNThreadDumpContent);
LOG.info("{}", localNNThreadDumpContent.toString());
isThreadDumpCaptured = true;
} catch (IOException e) {
LOG.warn("Can't get local NN thread dump due to " + e.getMessage());

View File

@ -28,8 +28,8 @@
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Options;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -57,8 +57,8 @@ public class DelegationTokenFetcher {
private static final String CANCEL = "cancel";
private static final String HELP = "help";
private static final String HELP_SHORT = "h";
private static final Log LOG = LogFactory
.getLog(DelegationTokenFetcher.class);
private static final Logger LOG = LoggerFactory
.getLogger(DelegationTokenFetcher.class);
private static final String PRINT = "print";
private static final String RENEW = "renew";
private static final String RENEWER = "renewer";

View File

@ -21,8 +21,8 @@
import java.io.PrintStream;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -43,7 +43,7 @@
@InterfaceAudience.Private
public class GetGroups extends GetGroupsBase {
private static final Log LOG = LogFactory.getLog(GetGroups.class);
private static final Logger LOG = LoggerFactory.getLogger(GetGroups.class);
static final String USAGE = "Usage: hdfs groups [username ...]";

View File

@ -19,8 +19,8 @@
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer;
@ -40,8 +40,8 @@ class OfflineEditsBinaryLoader implements OfflineEditsLoader {
private final boolean fixTxIds;
private final boolean recoveryMode;
private long nextTxId;
public static final Log LOG =
LogFactory.getLog(OfflineEditsBinaryLoader.class.getName());
public static final Logger LOG =
LoggerFactory.getLogger(OfflineEditsBinaryLoader.class.getName());
/**
* Constructor
@ -102,7 +102,7 @@ public void loadEdits() throws IOException {
}
visitor.close(null);
} finally {
IOUtils.cleanup(LOG, inputStream);
IOUtils.cleanupWithLogger(LOG, inputStream);
}
}
}

View File

@ -30,8 +30,8 @@
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.QueryStringDecoder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.util.StringUtils;
@ -58,7 +58,8 @@
* Implement the read-only WebHDFS API for fsimage.
*/
class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
public static final Log LOG = LogFactory.getLog(FSImageHandler.class);
public static final Logger LOG =
LoggerFactory.getLogger(FSImageHandler.class);
private final FSImageLoader image;
private final ChannelGroup activeChannels;

View File

@ -34,8 +34,8 @@
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;
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.XAttr;
import org.apache.hadoop.fs.permission.AclEntry;
@ -65,7 +65,8 @@
* file status of the namespace of the fsimage.
*/
class FSImageLoader {
public static final Log LOG = LogFactory.getLog(FSImageHandler.class);
public static final Logger LOG =
LoggerFactory.getLogger(FSImageHandler.class);
private final String[] stringTable;
// byte representation of inodes, sorted by id

View File

@ -54,8 +54,8 @@
import com.google.protobuf.ByteString;
import com.google.protobuf.TextFormat;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.permission.AclEntry;
@ -96,8 +96,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class OfflineImageReconstructor {
public static final Log LOG =
LogFactory.getLog(OfflineImageReconstructor.class);
public static final Logger LOG =
LoggerFactory.getLogger(OfflineImageReconstructor.class);
/**
* The output stream.
@ -1839,7 +1839,7 @@ public static void run(String inputPath, String outputPath)
new OfflineImageReconstructor(out, reader);
oir.processXml();
} finally {
IOUtils.cleanup(LOG, reader, fis, out, fout);
IOUtils.cleanupWithLogger(LOG, reader, fis, out, fout);
}
// Write the md5 file
MD5FileUtils.saveMD5File(new File(outputPath),

View File

@ -30,8 +30,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.IOUtils;
@ -44,7 +44,8 @@
*/
@InterfaceAudience.Private
public class OfflineImageViewer {
public static final Log LOG = LogFactory.getLog(OfflineImageViewer.class);
public static final Logger LOG =
LoggerFactory.getLogger(OfflineImageViewer.class);
private final static String usage =
"Usage: bin/hdfs oiv_legacy [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n"
@ -145,7 +146,7 @@ public void go() throws IOException {
LOG.error("Failed to load image file.");
}
}
IOUtils.cleanup(LOG, in, tracker);
IOUtils.cleanupWithLogger(LOG, in, tracker);
}
}

View File

@ -28,8 +28,8 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
@ -44,7 +44,8 @@
public class OfflineImageViewerPB {
private static final String HELP_OPT = "-h";
private static final String HELP_LONGOPT = "--help";
public static final Log LOG = LogFactory.getLog(OfflineImageViewerPB.class);
public static final Logger LOG =
LoggerFactory.getLogger(OfflineImageViewerPB.class);
private final static String usage = "Usage: bin/hdfs oiv [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n"
+ "Offline Image Viewer\n"

View File

@ -32,8 +32,8 @@
import io.netty.handler.codec.http.HttpResponseEncoder;
import io.netty.handler.codec.string.StringEncoder;
import io.netty.util.concurrent.GlobalEventExecutor;
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.CommonConfigurationKeysPublic;
import org.apache.hadoop.security.UserGroupInformation;
@ -47,7 +47,8 @@
* namespace.
*/
public class WebImageViewer implements Closeable {
public static final Log LOG = LogFactory.getLog(WebImageViewer.class);
public static final Logger LOG =
LoggerFactory.getLogger(WebImageViewer.class);
private Channel channel;
private InetSocketAddress address;

View File

@ -23,8 +23,8 @@
import java.io.FilterOutputStream;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.NativeIOException;
@ -46,7 +46,7 @@ public class AtomicFileOutputStream extends FilterOutputStream {
private static final String TMP_EXTENSION = ".tmp";
private final static Log LOG = LogFactory.getLog(
private final static Logger LOG = LoggerFactory.getLogger(
AtomicFileOutputStream.class);
private final File origFile;

Some files were not shown because too many files have changed in this diff Show More