HDFS-13695. Move logging to slf4j in HDFS package. Contributed by Ian Pickering.
This commit is contained in:
parent
b3161c4dd9
commit
eca1a4bfe9
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -649,7 +649,7 @@ public static Peer peerFromSocketAndKey(
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtilsClient.cleanup(null, peer);
|
||||
IOUtilsClient.cleanupWithLogger(LOG, peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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()) {
|
||||
|
@ -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 {
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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 */ }
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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";
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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>();
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
|
@ -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() {}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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";
|
||||
|
@ -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"
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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 {
|
||||
|
@ -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);
|
||||
|
@ -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()]);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
/**
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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}
|
||||
|
@ -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";
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -103,7 +103,7 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp)
|
||||
json.writeEndArray();
|
||||
json.writeEndObject();
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, json);
|
||||
IOUtils.cleanupWithLogger(LOG, json);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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 {
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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("");
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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());
|
||||
|
@ -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";
|
||||
|
@ -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 ...]";
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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),
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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"
|
||||
|
@ -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;
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user