Skip to content

Commit

Permalink
HDFS-12997. Move logging to slf4j in BlockPoolSliceStorage and Storag…
Browse files Browse the repository at this point in the history
…e. Contributed by Ajay Kumar.
  • Loading branch information
xiaoyuyao committed Feb 1, 2018
1 parent 6ca7204 commit b3ae11d
Show file tree
Hide file tree
Showing 6 changed files with 178 additions and 181 deletions.
Expand Up @@ -193,10 +193,9 @@ private static void purgeMatching(File dir, List<Pattern> patterns,
// /\d+/ in the regex itself.
long txid = Long.parseLong(matcher.group(1));
if (txid < minTxIdToKeep) {
LOG.info("Purging no-longer needed file " + txid);
LOG.info("Purging no-longer needed file {}", txid);
if (!f.delete()) {
LOG.warn("Unable to delete no-longer-needed data " +
f);
LOG.warn("Unable to delete no-longer-needed data {}", f);
}
break;
}
Expand All @@ -214,7 +213,7 @@ void format(NamespaceInfo nsInfo) throws IOException {
}
setStorageInfo(nsInfo);

LOG.info("Formatting journal " + sd + " with nsid: " + getNamespaceID());
LOG.info("Formatting journal {} with nsid: {}", sd, getNamespaceID());
// Unlock the directory before formatting, because we will
// re-analyze it after format(). The analyzeStorage() call
// below is reponsible for re-locking it. This is a no-op
Expand Down Expand Up @@ -278,7 +277,7 @@ void checkConsistentNamespace(NamespaceInfo nsInfo)
}

public void close() throws IOException {
LOG.info("Closing journal storage for " + sd);
LOG.info("Closing journal storage for {}", sd);
unlockAll();
}

Expand Down
Expand Up @@ -35,8 +35,6 @@
import java.util.concurrent.CopyOnWriteArrayList;

import org.apache.commons.io.FileUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
Expand All @@ -53,7 +51,8 @@

import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


/**
Expand All @@ -76,7 +75,9 @@
*/
@InterfaceAudience.Private
public abstract class Storage extends StorageInfo {
public static final Log LOG = LogFactory.getLog(Storage.class.getName());

public static final Logger LOG = LoggerFactory
.getLogger(Storage.class.getName());

// last layout version that did not support upgrades
public static final int LAST_PRE_UPGRADE_LAYOUT_VERSION = -3;
Expand Down Expand Up @@ -396,7 +397,7 @@ public long getDirecorySize() {
return FileUtils.sizeOfDirectory(root);
}
} catch (Exception e) {
LOG.warn("Failed to get directory size :" + root, e);
LOG.warn("Failed to get directory size : {}", root, e);
}
return 0;
}
Expand Down Expand Up @@ -427,7 +428,7 @@ public void clearDirectory() throws IOException {
}
if (curDir.exists()) {
File[] files = FileUtil.listFiles(curDir);
LOG.info("Will remove files: " + Arrays.toString(files));
LOG.info("Will remove files: {}", Arrays.toString(files));
if (!(FileUtil.fullyDelete(curDir)))
throw new IOException("Cannot remove current directory: " + curDir);
}
Expand Down Expand Up @@ -650,25 +651,25 @@ public StorageState analyzeStorage(StartupOption startOpt, Storage storage,
// storage directory does not exist
if (startOpt != StartupOption.FORMAT &&
startOpt != StartupOption.HOTSWAP) {
LOG.warn("Storage directory " + rootPath + " does not exist");
LOG.warn("Storage directory {} does not exist", rootPath);
return StorageState.NON_EXISTENT;
}
LOG.info(rootPath + " does not exist. Creating ...");
LOG.info("{} does not exist. Creating ...", rootPath);
if (!root.mkdirs())
throw new IOException("Cannot create directory " + rootPath);
hadMkdirs = true;
}
// or is inaccessible
if (!root.isDirectory()) {
LOG.warn(rootPath + "is not a directory");
LOG.warn("{} is not a directory", rootPath);
return StorageState.NON_EXISTENT;
}
if (!FileUtil.canWrite(root)) {
LOG.warn("Cannot access storage directory " + rootPath);
LOG.warn("Cannot access storage directory {}", rootPath);
return StorageState.NON_EXISTENT;
}
} catch(SecurityException ex) {
LOG.warn("Cannot access storage directory " + rootPath, ex);
LOG.warn("Cannot access storage directory {}", rootPath, ex);
return StorageState.NON_EXISTENT;
}

Expand Down Expand Up @@ -770,43 +771,43 @@ public void doRecover(StorageState curState) throws IOException {
String rootPath = root.getCanonicalPath();
switch(curState) {
case COMPLETE_UPGRADE: // mv previous.tmp -> previous
LOG.info("Completing previous upgrade for storage directory "
+ rootPath);
LOG.info("Completing previous upgrade for storage directory {}",
rootPath);
rename(getPreviousTmp(), getPreviousDir());
return;
case RECOVER_UPGRADE: // mv previous.tmp -> current
LOG.info("Recovering storage directory " + rootPath
+ " from previous upgrade");
LOG.info("Recovering storage directory {} from previous upgrade",
rootPath);
if (curDir.exists())
deleteDir(curDir);
rename(getPreviousTmp(), curDir);
return;
case COMPLETE_ROLLBACK: // rm removed.tmp
LOG.info("Completing previous rollback for storage directory "
+ rootPath);
LOG.info("Completing previous rollback for storage directory {}",
rootPath);
deleteDir(getRemovedTmp());
return;
case RECOVER_ROLLBACK: // mv removed.tmp -> current
LOG.info("Recovering storage directory " + rootPath
+ " from previous rollback");
LOG.info("Recovering storage directory {} from previous rollback",
rootPath);
rename(getRemovedTmp(), curDir);
return;
case COMPLETE_FINALIZE: // rm finalized.tmp
LOG.info("Completing previous finalize for storage directory "
+ rootPath);
LOG.info("Completing previous finalize for storage directory {}",
rootPath);
deleteDir(getFinalizedTmp());
return;
case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
LOG.info("Completing previous checkpoint for storage directory "
+ rootPath);
LOG.info("Completing previous checkpoint for storage directory {}",
rootPath);
File prevCkptDir = getPreviousCheckpoint();
if (prevCkptDir.exists())
deleteDir(prevCkptDir);
rename(getLastCheckpointTmp(), prevCkptDir);
return;
case RECOVER_CHECKPOINT: // mv lastcheckpoint.tmp -> current
LOG.info("Recovering storage directory " + rootPath
+ " from failed checkpoint");
LOG.info("Recovering storage directory {} from failed checkpoint",
rootPath);
if (curDir.exists())
deleteDir(curDir);
rename(getLastCheckpointTmp(), curDir);
Expand Down Expand Up @@ -860,12 +861,12 @@ public boolean isShared() {
*/
public void lock() throws IOException {
if (isShared()) {
LOG.info("Locking is disabled for " + this.root);
LOG.info("Locking is disabled for {}", this.root);
return;
}
FileLock newLock = tryLock();
if (newLock == null) {
String msg = "Cannot lock storage " + this.root
String msg = "Cannot lock storage " + this.root
+ ". The directory is already locked";
LOG.info(msg);
throw new IOException(msg);
Expand Down Expand Up @@ -897,22 +898,22 @@ FileLock tryLock() throws IOException {
try {
res = file.getChannel().tryLock();
if (null == res) {
LOG.error("Unable to acquire file lock on path " + lockF.toString());
LOG.error("Unable to acquire file lock on path {}", lockF);
throw new OverlappingFileLockException();
}
file.write(jvmName.getBytes(Charsets.UTF_8));
LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName);
LOG.info("Lock on {} acquired by nodename {}", lockF, jvmName);
} catch(OverlappingFileLockException oe) {
// Cannot read from the locked file on Windows.
String lockingJvmName = Path.WINDOWS ? "" : (" " + file.readLine());
LOG.error("It appears that another node " + lockingJvmName
+ " has already locked the storage directory: " + root, oe);
LOG.error("It appears that another node {} has already locked the "
+ "storage directory: {}", lockingJvmName, root, oe);
file.close();
return null;
} catch(IOException e) {
LOG.error("Failed to acquire lock on " + lockF
+ ". If this storage directory is mounted via NFS, "
+ "ensure that the appropriate nfs lock services are running.", e);
LOG.error("Failed to acquire lock on {}. If this storage directory is"
+ " mounted via NFS, ensure that the appropriate nfs lock services"
+ " are running.", lockF, e);
file.close();
throw e;
}
Expand Down Expand Up @@ -1331,10 +1332,8 @@ public static void nativeCopyFileUnbuffered(File srcFile, File destFile,
}
if (preserveFileDate) {
if (destFile.setLastModified(srcFile.lastModified()) == false) {
if (LOG.isDebugEnabled()) {
LOG.debug("Failed to preserve last modified date from'" + srcFile
+ "' to '" + destFile + "'");
}
LOG.debug("Failed to preserve last modified date from'{}' to '{}'",
srcFile, destFile);
}
}
}
Expand Down

0 comments on commit b3ae11d

Please sign in to comment.