HDFS-12997. Move logging to slf4j in BlockPoolSliceStorage and Storage. Contributed by Ajay Kumar.

This commit is contained in:
Xiaoyu Yao 2018-01-31 23:10:54 -08:00
parent 6ca7204ceb
commit b3ae11d597
6 changed files with 178 additions and 181 deletions

View File

@ -193,10 +193,9 @@ class JNStorage extends Storage {
// /\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;
}
@ -214,7 +213,7 @@ class JNStorage extends Storage {
}
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
@ -278,7 +277,7 @@ class JNStorage extends Storage {
}
public void close() throws IOException {
LOG.info("Closing journal storage for " + sd);
LOG.info("Closing journal storage for {}", sd);
unlockAll();
}

View File

@ -35,8 +35,6 @@ import java.util.Properties;
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;
@ -53,7 +51,8 @@ import org.apache.hadoop.util.VersionInfo;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
@ -76,7 +75,9 @@ import com.google.common.base.Preconditions;
*/
@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;
@ -396,7 +397,7 @@ public abstract class Storage extends StorageInfo {
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;
}
@ -427,7 +428,7 @@ public abstract class Storage extends StorageInfo {
}
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);
}
@ -650,25 +651,25 @@ public abstract class Storage extends StorageInfo {
// 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;
}
@ -770,43 +771,43 @@ public abstract class Storage extends StorageInfo {
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);
@ -860,12 +861,12 @@ public abstract class Storage extends StorageInfo {
*/
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);
@ -897,22 +898,22 @@ public abstract class Storage extends StorageInfo {
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;
}
@ -1331,10 +1332,8 @@ public abstract class Storage extends StorageInfo {
}
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);
}
}
}

View File

@ -159,17 +159,15 @@ public class BlockPoolSliceStorage extends Storage {
case NORMAL:
break;
case NON_EXISTENT:
LOG.info("Block pool storage directory for location " + location +
" and block pool id " + nsInfo.getBlockPoolID() +
" does not exist");
LOG.info("Block pool storage directory for location {} and block pool"
+ " id {} does not exist", location, nsInfo.getBlockPoolID());
throw new IOException("Storage directory for location " + location +
" and block pool id " + nsInfo.getBlockPoolID() +
" does not exist");
case NOT_FORMATTED: // format
LOG.info("Block pool storage directory for location " + location +
" and block pool id " + nsInfo.getBlockPoolID()
+ " is not formatted for " + nsInfo.getBlockPoolID()
+ ". Formatting ...");
LOG.info("Block pool storage directory for location {} and block pool"
+ " id {} is not formatted. Formatting ...", location,
nsInfo.getBlockPoolID());
format(sd, nsInfo);
break;
default: // recovery part is common
@ -227,8 +225,8 @@ public class BlockPoolSliceStorage extends Storage {
nsInfo, location, startOpt, callables, conf);
succeedDirs.add(sd);
} catch (IOException e) {
LOG.warn("Failed to analyze storage directories for block pool "
+ nsInfo.getBlockPoolID(), e);
LOG.warn("Failed to analyze storage directories for block pool {}",
nsInfo.getBlockPoolID(), e);
throw e;
}
return succeedDirs;
@ -250,7 +248,8 @@ public class BlockPoolSliceStorage extends Storage {
StorageLocation location, StartupOption startOpt,
List<Callable<StorageDirectory>> callables, Configuration conf)
throws IOException {
LOG.info("Analyzing storage directories for bpid " + nsInfo.getBlockPoolID());
LOG.info("Analyzing storage directories for bpid {}", nsInfo
.getBlockPoolID());
final List<StorageDirectory> loaded = loadBpStorageDirectories(
nsInfo, location, startOpt, callables, conf);
for (StorageDirectory sd : loaded) {
@ -278,8 +277,8 @@ public class BlockPoolSliceStorage extends Storage {
* @throws IOException Signals that an I/O exception has occurred.
*/
private void format(StorageDirectory bpSdir, NamespaceInfo nsInfo) throws IOException {
LOG.info("Formatting block pool " + blockpoolID + " directory "
+ bpSdir.getCurrentDir());
LOG.info("Formatting block pool {} directory {}", blockpoolID, bpSdir
.getCurrentDir());
bpSdir.clearDirectory(); // create directory
this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
this.cTime = nsInfo.getCTime();
@ -295,7 +294,7 @@ public class BlockPoolSliceStorage extends Storage {
*/
void remove(File absPathToRemove) {
Preconditions.checkArgument(absPathToRemove.isAbsolute());
LOG.info("Removing block level storage: " + absPathToRemove);
LOG.info("Removing block level storage: {}", absPathToRemove);
for (Iterator<StorageDirectory> it = getStorageDirs().iterator();
it.hasNext(); ) {
StorageDirectory sd = it.next();
@ -375,7 +374,7 @@ public class BlockPoolSliceStorage extends Storage {
// during rolling upgrade rollback. They are deleted during rolling
// upgrade downgrade.
int restored = restoreBlockFilesFromTrash(getTrashRootDir(sd));
LOG.info("Restored " + restored + " block files from trash.");
LOG.info("Restored {} block files from trash.", restored);
}
readProperties(sd);
checkVersionUpgradable(this.layoutVersion);
@ -399,9 +398,9 @@ public class BlockPoolSliceStorage extends Storage {
}
if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
int restored = restoreBlockFilesFromTrash(getTrashRootDir(sd));
LOG.info("Restored " + restored + " block files from trash " +
"before the layout upgrade. These blocks will be moved to " +
"the previous directory during the upgrade");
LOG.info("Restored {} block files from trash " +
"before the layout upgrade. These blocks will be moved to " +
"the previous directory during the upgrade", restored);
}
if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION
|| this.cTime < nsInfo.getCTime()) {
@ -448,11 +447,10 @@ public class BlockPoolSliceStorage extends Storage {
return;
}
final int oldLV = getLayoutVersion();
LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
+ ".\n old LV = " + oldLV
+ "; old CTime = " + this.getCTime()
+ ".\n new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
+ "; new CTime = " + nsInfo.getCTime());
LOG.info("Upgrading block pool storage directory {}.\n old LV = {}; old"
+ " CTime = {}.\n new LV = {}; new CTime = {}",
bpSd.getRoot(), oldLV, this.getCTime(), HdfsServerConstants
.DATANODE_LAYOUT_VERSION, nsInfo.getCTime());
// get <SD>/previous directory
String dnRoot = getDataNodeStorageRoot(bpSd.getRoot().getCanonicalPath());
StorageDirectory dnSdStorage = new StorageDirectory(new File(dnRoot));
@ -508,7 +506,7 @@ public class BlockPoolSliceStorage extends Storage {
// 4.rename <SD>/current/<bpid>/previous.tmp to
// <SD>/current/<bpid>/previous
rename(bpTmpDir, bpPrevDir);
LOG.info("Upgrade of " + name + " is complete");
LOG.info("Upgrade of {} is complete", name);
}
/**
@ -569,8 +567,8 @@ public class BlockPoolSliceStorage extends Storage {
// Failsafe - we should not hit this case but let's make sure
// we never overwrite a newer version of a block file with an
// older version.
LOG.info("Not overwriting " + newChild + " with smaller file from " +
"trash directory. This message can be safely ignored.");
LOG.info("Not overwriting {} with smaller file from " +
"trash directory. This message can be safely ignored.", newChild);
} else if (!child.renameTo(newChild)) {
throw new IOException("Failed to rename " + child + " to " + newChild);
} else {
@ -616,10 +614,10 @@ public class BlockPoolSliceStorage extends Storage {
+ " is newer than the namespace state: LV = "
+ HdfsServerConstants.DATANODE_LAYOUT_VERSION + " CTime = " + nsInfo.getCTime());
}
LOG.info("Rolling back storage directory " + bpSd.getRoot()
+ ".\n target LV = " + nsInfo.getLayoutVersion()
+ "; target CTime = " + nsInfo.getCTime());
LOG.info("Rolling back storage directory {}.\n target LV = {}; target "
+ "CTime = {}", bpSd.getRoot(), nsInfo.getLayoutVersion(),
nsInfo.getCTime());
File tmpDir = bpSd.getRemovedTmp();
assert !tmpDir.exists() : "removed.tmp directory must not exist.";
// 1. rename current to tmp
@ -632,7 +630,7 @@ public class BlockPoolSliceStorage extends Storage {
// 3. delete removed.tmp dir
deleteDir(tmpDir);
LOG.info("Rollback of " + bpSd.getRoot() + " is complete");
LOG.info("Rollback of {} is complete", bpSd.getRoot());
}
/*
@ -651,9 +649,9 @@ public class BlockPoolSliceStorage extends Storage {
return; // already finalized
}
final String dataDirPath = bpSd.getRoot().getCanonicalPath();
LOG.info("Finalizing upgrade for storage directory " + dataDirPath
+ ".\n cur LV = " + this.getLayoutVersion() + "; cur CTime = "
+ this.getCTime());
LOG.info("Finalizing upgrade for storage directory {}.\n cur LV = {}; "
+ "cur CTime = {}", dataDirPath, this.getLayoutVersion(),
this.getCTime());
assert bpSd.getCurrentDir().exists() : "Current directory must exist.";
// rename previous to finalized.tmp
@ -667,9 +665,9 @@ public class BlockPoolSliceStorage extends Storage {
try {
deleteDir(tmpDir);
} catch (IOException ex) {
LOG.error("Finalize upgrade for " + dataDirPath + " failed.", ex);
LOG.error("Finalize upgrade for {} failed.", dataDirPath, ex);
}
LOG.info("Finalize upgrade for " + dataDirPath + " is complete.");
LOG.info("Finalize upgrade for {} is complete.", dataDirPath);
}
@Override
@ -695,8 +693,8 @@ public class BlockPoolSliceStorage extends Storage {
diskLayoutVersion, hardLink, conf);
DataStorage.linkBlocks(fromDir, toDir, DataStorage.STORAGE_DIR_RBW,
diskLayoutVersion, hardLink, conf);
LOG.info("Linked blocks from " + fromDir + " to " + toDir + ". "
+ hardLink.linkStats.report());
LOG.info("Linked blocks from {} to {}. {}", fromDir, toDir,
hardLink.linkStats.report());
}
/**
@ -763,7 +761,7 @@ public class BlockPoolSliceStorage extends Storage {
File blockFile = new File(blockURI);
return getTrashDirectory(blockFile);
} catch (IllegalArgumentException e) {
LOG.warn("Failed to get block file for replica " + info, e);
LOG.warn("Failed to get block file for replica {}", info, e);
}
return null;
@ -791,7 +789,7 @@ public class BlockPoolSliceStorage extends Storage {
String getRestoreDirectory(File blockFile) {
Matcher matcher = BLOCK_POOL_TRASH_PATH_PATTERN.matcher(blockFile.getParent());
String restoreDirectory = matcher.replaceFirst("$1$2" + STORAGE_DIR_CURRENT + "$4");
LOG.info("Restoring " + blockFile + " to " + restoreDirectory);
LOG.info("Restoring {} to {}", blockFile, restoreDirectory);
return restoreDirectory;
}
@ -804,7 +802,7 @@ public class BlockPoolSliceStorage extends Storage {
File trashRoot = getTrashRootDir(sd);
if (trashRoot.exists() && sd.getPreviousDir().exists()) {
LOG.error("Trash and PreviousDir shouldn't both exist for storage "
+ "directory " + sd);
+ "directory {}", sd);
assert false;
} else {
trashRoots.add(trashRoot);
@ -817,7 +815,7 @@ public class BlockPoolSliceStorage extends Storage {
public void run() {
for(File trashRoot : trashRoots){
FileUtil.fullyDelete(trashRoot);
LOG.info("Cleared trash for storage directory " + trashRoot);
LOG.info("Cleared trash for storage directory {}", trashRoot);
}
}
@ -860,9 +858,9 @@ public class BlockPoolSliceStorage extends Storage {
File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
if (!storagesWithRollingUpgradeMarker.contains(bpRoot.toString())) {
if (!markerFile.exists() && markerFile.createNewFile()) {
LOG.info("Created " + markerFile);
LOG.info("Created {}", markerFile);
} else {
LOG.info(markerFile + " already exists.");
LOG.info("{} already exists.", markerFile);
}
storagesWithRollingUpgradeMarker.add(bpRoot.toString());
storagesWithoutRollingUpgradeMarker.remove(bpRoot.toString());
@ -885,10 +883,10 @@ public class BlockPoolSliceStorage extends Storage {
File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
if (!storagesWithoutRollingUpgradeMarker.contains(bpRoot.toString())) {
if (markerFile.exists()) {
LOG.info("Deleting " + markerFile);
LOG.info("Deleting {}", markerFile);
doFinalize(sd.getCurrentDir());
if (!markerFile.delete()) {
LOG.warn("Failed to delete " + markerFile);
LOG.warn("Failed to delete {}", markerFile);
}
}
storagesWithoutRollingUpgradeMarker.add(bpRoot.toString());

View File

@ -157,8 +157,8 @@ public class DataStorage extends Storage {
}
if (oldStorageID == null || regenerateStorageIds) {
sd.setStorageUuid(DatanodeStorage.generateUuid());
LOG.info("Generated new storageID " + sd.getStorageUuid() +
" for directory " + sd.getRoot() +
LOG.info("Generated new storageID {} for directory {} {}", sd
.getStorageUuid(), sd.getRoot(),
(oldStorageID == null ? "" : (" to replace " + oldStorageID)));
return true;
}
@ -173,7 +173,7 @@ public class DataStorage extends Storage {
public void enableTrash(String bpid) {
if (trashEnabledBpids.add(bpid)) {
getBPStorage(bpid).stopTrashCleaner();
LOG.info("Enabled trash for bpid " + bpid);
LOG.info("Enabled trash for bpid {}", bpid);
}
}
@ -181,7 +181,7 @@ public class DataStorage extends Storage {
if (trashEnabledBpids.contains(bpid)) {
getBPStorage(bpid).clearTrash();
trashEnabledBpids.remove(bpid);
LOG.info("Cleared trash for bpid " + bpid);
LOG.info("Cleared trash for bpid {}", bpid);
}
}
@ -275,14 +275,12 @@ public class DataStorage extends Storage {
case NORMAL:
break;
case NON_EXISTENT:
LOG.info("Storage directory with location " + location
+ " does not exist");
LOG.info("Storage directory with location {} does not exist", location);
throw new IOException("Storage directory with location " + location
+ " does not exist");
case NOT_FORMATTED: // format
LOG.info("Storage directory with location " + location
+ " is not formatted for namespace " + nsInfo.getNamespaceID()
+ ". Formatting...");
LOG.info("Storage directory with location {} is not formatted for "
+ "namespace {}. Formatting...", location, nsInfo.getNamespaceID());
format(sd, nsInfo, datanode.getDatanodeUuid(), datanode.getConf());
break;
default: // recovery part is common
@ -325,8 +323,8 @@ public class DataStorage extends Storage {
StorageLocation location, List<NamespaceInfo> nsInfos)
throws IOException {
if (containsStorageDir(location)) {
final String errorMessage = "Storage directory is in use";
LOG.warn(errorMessage + ".");
final String errorMessage = "Storage directory is in use.";
LOG.warn(errorMessage);
throw new IOException(errorMessage);
}
@ -353,8 +351,8 @@ public class DataStorage extends Storage {
throw new HadoopIllegalArgumentException(key + " = " + n + " < 1");
}
final int min = Math.min(n, dataDirs);
LOG.info("Using " + min + " threads to upgrade data directories ("
+ key + "=" + n + ", dataDirs=" + dataDirs + ")");
LOG.info("Using {} threads to upgrade data directories ({}={}, "
+ "dataDirs={})", min, key, n, dataDirs);
return min;
}
@ -417,22 +415,22 @@ public class DataStorage extends Storage {
}
}
} catch (IOException e) {
LOG.warn("Failed to add storage directory " + dataDir, e);
LOG.warn("Failed to add storage directory {}", dataDir, e);
}
} else {
LOG.info("Storage directory " + dataDir + " has already been used.");
LOG.info("Storage directory {} has already been used.", dataDir);
success.add(dataDir);
}
}
if (!tasks.isEmpty()) {
LOG.info("loadDataStorage: " + tasks.size() + " upgrade tasks");
LOG.info("loadDataStorage: {} upgrade tasks", tasks.size());
for(UpgradeTask t : tasks) {
try {
addStorageDir(t.future.get());
success.add(t.dataDir);
} catch (ExecutionException e) {
LOG.warn("Failed to upgrade storage directory " + t.dataDir, e);
LOG.warn("Failed to upgrade storage directory {}", t.dataDir, e);
} catch (InterruptedException e) {
throw DFSUtilClient.toInterruptedIOException("Task interrupted", e);
}
@ -465,19 +463,19 @@ public class DataStorage extends Storage {
}
}
} catch (IOException e) {
LOG.warn("Failed to add storage directory " + dataDir
+ " for block pool " + bpid, e);
LOG.warn("Failed to add storage directory {} for block pool {}",
dataDir, bpid, e);
}
}
if (!tasks.isEmpty()) {
LOG.info("loadBlockPoolSliceStorage: " + tasks.size() + " upgrade tasks");
LOG.info("loadBlockPoolSliceStorage: {} upgrade tasks", tasks.size());
for(UpgradeTask t : tasks) {
try {
success.add(t.future.get());
} catch (ExecutionException e) {
LOG.warn("Failed to upgrade storage directory " + t.dataDir
+ " for block pool " + bpid, e);
LOG.warn("Failed to upgrade storage directory {} for block pool {}",
t.dataDir, bpid, e);
} catch (InterruptedException e) {
throw DFSUtilClient.toInterruptedIOException("Task interrupted", e);
}
@ -491,7 +489,7 @@ public class DataStorage extends Storage {
* Remove storage dirs from DataStorage. All storage dirs are removed even when the
* IOException is thrown.
*
* @param dirsToRemove a set of storage directories to be removed.
* @param storageLocations a set of storage directories to be removed.
* @throws IOException if I/O error when unlocking storage directory.
*/
synchronized void removeVolumes(
@ -521,9 +519,8 @@ public class DataStorage extends Storage {
try {
sd.unlock();
} catch (IOException e) {
LOG.warn(String.format(
"I/O error attempting to unlock storage directory %s.",
sd.getRoot()), e);
LOG.warn("I/O error attempting to unlock storage directory {}.",
sd.getRoot(), e);
errorMsgBuilder.append(String.format("Failed to remove %s: %s%n",
sd.getRoot(), e.getMessage()));
}
@ -672,7 +669,7 @@ public class DataStorage extends Storage {
try (RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
FileLock oldLock = oldFile.getChannel().tryLock()) {
if (null == oldLock) {
LOG.error("Unable to acquire file lock on path " + oldF.toString());
LOG.error("Unable to acquire file lock on path {}", oldF);
throw new OverlappingFileLockException();
}
oldFile.seek(0);
@ -793,11 +790,10 @@ public class DataStorage extends Storage {
final List<Callable<StorageDirectory>> callables,
final Configuration conf) throws IOException {
final int oldLV = getLayoutVersion();
LOG.info("Upgrading storage directory " + sd.getRoot()
+ ".\n old LV = " + oldLV
+ "; old CTime = " + this.getCTime()
+ ".\n new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
+ "; new CTime = " + nsInfo.getCTime());
LOG.info("Upgrading storage directory {}.\n old LV = {}; old CTime = {}"
+ ".\n new LV = {}; new CTime = {}", sd.getRoot(), oldLV,
this.getCTime(), HdfsServerConstants.DATANODE_LAYOUT_VERSION,
nsInfo.getCTime());
final File curDir = sd.getCurrentDir();
final File prevDir = sd.getPreviousDir();
@ -850,15 +846,15 @@ public class DataStorage extends Storage {
// 5. Rename <SD>/previous.tmp to <SD>/previous
rename(tmpDir, prevDir);
LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
LOG.info("Upgrade of {} is complete", sd.getRoot());
}
void upgradeProperties(StorageDirectory sd, Configuration conf)
throws IOException {
createStorageID(sd, layoutVersion, conf);
LOG.info("Updating layout version from " + layoutVersion
+ " to " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
+ " for storage " + sd.getRoot());
LOG.info("Updating layout version from {} to {} for storage {}",
layoutVersion, HdfsServerConstants.DATANODE_LAYOUT_VERSION,
sd.getRoot());
layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
writeProperties(sd);
}
@ -914,9 +910,8 @@ public class DataStorage extends Storage {
HdfsServerConstants.DATANODE_LAYOUT_VERSION)) {
readProperties(sd, HdfsServerConstants.DATANODE_LAYOUT_VERSION);
writeProperties(sd);
LOG.info("Layout version rolled back to "
+ HdfsServerConstants.DATANODE_LAYOUT_VERSION + " for storage "
+ sd.getRoot());
LOG.info("Layout version rolled back to {} for storage {}",
HdfsServerConstants.DATANODE_LAYOUT_VERSION, sd.getRoot());
}
return;
}
@ -933,9 +928,9 @@ public class DataStorage extends Storage {
+ " is newer than the namespace state: LV = "
+ HdfsServerConstants.DATANODE_LAYOUT_VERSION + " CTime = "
+ nsInfo.getCTime());
LOG.info("Rolling back storage directory " + sd.getRoot()
+ ".\n target LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
+ "; target CTime = " + nsInfo.getCTime());
LOG.info("Rolling back storage directory {}.\n target LV = {}; target "
+ "CTime = {}", sd.getRoot(),
HdfsServerConstants.DATANODE_LAYOUT_VERSION, nsInfo.getCTime());
File tmpDir = sd.getRemovedTmp();
assert !tmpDir.exists() : "removed.tmp directory must not exist.";
// rename current to tmp
@ -946,7 +941,7 @@ public class DataStorage extends Storage {
rename(prevDir, curDir);
// delete tmp dir
deleteDir(tmpDir);
LOG.info("Rollback of " + sd.getRoot() + " is complete");
LOG.info("Rollback of {} is complete", sd.getRoot());
}
/**
@ -964,10 +959,9 @@ public class DataStorage extends Storage {
return; // already discarded
final String dataDirPath = sd.getRoot().getCanonicalPath();
LOG.info("Finalizing upgrade for storage directory "
+ dataDirPath
+ ".\n cur LV = " + this.getLayoutVersion()
+ "; cur CTime = " + this.getCTime());
LOG.info("Finalizing upgrade for storage directory {}.\n cur LV = {}; "
+ "cur CTime = {}", dataDirPath, this.getLayoutVersion(), this
.getCTime());
assert sd.getCurrentDir().exists() : "Current directory must exist.";
final File tmpDir = sd.getFinalizedTmp();//finalized.tmp directory
final File bbwDir = new File(sd.getRoot(), Storage.STORAGE_1_BBW);
@ -1055,8 +1049,8 @@ public class DataStorage extends Storage {
diskLayoutVersion, hardLink, conf);
}
}
LOG.info("Linked blocks from " + fromDir + " to " + toDir + ". "
+ hardLink.linkStats.report());
LOG.info("Linked blocks from {} to {}. {}", fromDir, toDir, hardLink
.linkStats.report());
}
private static class LinkArgs {
@ -1077,7 +1071,7 @@ public class DataStorage extends Storage {
private static void linkBlocks(File from, File to, int oldLV,
HardLink hl, Configuration conf) throws IOException {
LOG.info("Start linking block files from " + from + " to " + to);
LOG.info("Start linking block files from {} to {}", from, to);
boolean upgradeToIdBasedLayout = false;
// If we are upgrading from a version older than the one where we introduced
// block ID-based layout (32x32) AND we're working with the finalized
@ -1098,8 +1092,8 @@ public class DataStorage extends Storage {
final ArrayList<LinkArgs> duplicates =
findDuplicateEntries(idBasedLayoutSingleLinks);
if (!duplicates.isEmpty()) {
LOG.error("There are " + duplicates.size() + " duplicate block " +
"entries within the same volume.");
LOG.error("There are {} duplicate block " +
"entries within the same volume.", duplicates.size());
removeDuplicateEntries(idBasedLayoutSingleLinks, duplicates);
}
@ -1241,8 +1235,8 @@ public class DataStorage extends Storage {
}
}
if (!found) {
LOG.warn("Unexpectedly low genstamp on " +
duplicate.src.getAbsolutePath() + ".");
LOG.warn("Unexpectedly low genstamp on {}.",
duplicate.src.getAbsolutePath());
iter.remove();
}
}
@ -1265,13 +1259,13 @@ public class DataStorage extends Storage {
long blockLength = duplicate.src.length();
long prevBlockLength = prevLongest.src.length();
if (blockLength < prevBlockLength) {
LOG.warn("Unexpectedly short length on " +
duplicate.src.getAbsolutePath() + ".");
LOG.warn("Unexpectedly short length on {}.",
duplicate.src.getAbsolutePath());
continue;
}
if (blockLength > prevBlockLength) {
LOG.warn("Unexpectedly short length on " +
prevLongest.src.getAbsolutePath() + ".");
LOG.warn("Unexpectedly short length on {}.",
prevLongest.src.getAbsolutePath());
}
longestBlockFiles.put(blockId, duplicate);
}
@ -1286,7 +1280,7 @@ public class DataStorage extends Storage {
continue; // file has no duplicates
}
if (!bestDuplicate.src.getParent().equals(args.src.getParent())) {
LOG.warn("Discarding " + args.src.getAbsolutePath() + ".");
LOG.warn("Discarding {}.", args.src.getAbsolutePath());
iter.remove();
}
}

View File

@ -202,7 +202,7 @@ public class NNStorage extends Storage implements Closeable,
return false;
}
} finally {
IOUtils.cleanup(LOG, oldFile);
IOUtils.cleanupWithLogger(LOG, oldFile);
}
return true;
}
@ -220,7 +220,7 @@ public class NNStorage extends Storage implements Closeable,
* @param val Whether restoration attempt should be made.
*/
void setRestoreFailedStorage(boolean val) {
LOG.warn("set restore failed storage to " + val);
LOG.warn("set restore failed storage to {}", val);
restoreFailedStorage=val;
}
@ -243,14 +243,14 @@ public class NNStorage extends Storage implements Closeable,
/* We don't want more than one thread trying to restore at a time */
synchronized (this.restorationLock) {
LOG.info("NNStorage.attemptRestoreRemovedStorage: check removed(failed) "+
"storage. removedStorages size = " + removedStorageDirs.size());
"storage. removedStorages size = {}", removedStorageDirs.size());
for (StorageDirectory sd : this.removedStorageDirs) {
File root = sd.getRoot();
LOG.info("currently disabled dir " + root.getAbsolutePath() +
"; type=" + sd.getStorageDirType()
+ ";canwrite=" + FileUtil.canWrite(root));
LOG.info("currently disabled dir {}; type={} ;canwrite={}", root
.getAbsolutePath(), sd.getStorageDirType(),
FileUtil.canWrite(root));
if (root.exists() && FileUtil.canWrite(root)) {
LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
LOG.info("restoring dir {}", sd.getRoot().getAbsolutePath());
this.addStorageDir(sd); // restore
this.removedStorageDirs.remove(sd);
}
@ -510,7 +510,7 @@ public class NNStorage extends Storage implements Closeable,
writeTransactionIdFile(sd, txid);
} catch(IOException e) {
// Close any edits stream associated with this dir and remove directory
LOG.warn("writeTransactionIdToStorage failed on " + sd,
LOG.warn("writeTransactionIdToStorage failed on {}", sd,
e);
reportErrorsOnDirectory(sd);
}
@ -580,8 +580,8 @@ public class NNStorage extends Storage implements Closeable,
writeProperties(sd);
writeTransactionIdFile(sd, 0);
LOG.info("Storage directory " + sd.getRoot()
+ " has been successfully formatted.");
LOG.info("Storage directory {} has been successfully formatted.",
sd.getRoot());
}
/**
@ -861,26 +861,30 @@ public class NNStorage extends Storage implements Closeable,
* @param sd A storage directory to mark as errored.
*/
private void reportErrorsOnDirectory(StorageDirectory sd) {
LOG.error("Error reported on storage directory " + sd);
LOG.error("Error reported on storage directory {}", sd);
String lsd = listStorageDirectories();
LOG.debug("current list of storage dirs:" + lsd);
if(LOG.isDebugEnabled()){
String lsd = listStorageDirectories();
LOG.debug("current list of storage dirs:{}", lsd);
}
LOG.warn("About to remove corresponding storage: "
+ sd.getRoot().getAbsolutePath());
LOG.warn("About to remove corresponding storage: {}", sd.getRoot()
.getAbsolutePath());
try {
sd.unlock();
} catch (Exception e) {
LOG.warn("Unable to unlock bad storage directory: "
+ sd.getRoot().getPath(), e);
LOG.warn("Unable to unlock bad storage directory: {}", sd.getRoot()
.getPath(), e);
}
if (getStorageDirs().remove(sd)) {
this.removedStorageDirs.add(sd);
}
lsd = listStorageDirectories();
LOG.debug("at the end current list of storage dirs:" + lsd);
if(LOG.isDebugEnabled()){
String lsd = listStorageDirectories();
LOG.debug("at the end current list of storage dirs:{}", lsd);
}
}
/**
@ -911,11 +915,12 @@ public class NNStorage extends Storage implements Closeable,
// clusterid.
if (startOpt.getClusterId() != null
&& !startOpt.getClusterId().equals(getClusterID())) {
LOG.warn("Clusterid mismatch - current clusterid: " + getClusterID()
+ ", Ignoring given clusterid: " + startOpt.getClusterId());
LOG.warn("Clusterid mismatch - current clusterid: {}, Ignoring "
+ "given clusterid: {}", getClusterID(),
startOpt.getClusterId());
}
}
LOG.info("Using clusterid: " + getClusterID());
LOG.info("Using clusterid: {}", getClusterID());
}
}
@ -984,14 +989,14 @@ public class NNStorage extends Storage implements Closeable,
} else {
cid = props.getProperty("clusterID");
}
LOG.info("current cluster id for sd="+sd.getCurrentDir() +
";lv=" + layoutVersion + ";cid=" + cid);
LOG.info("current cluster id for sd={};lv={};"
+ "cid={}", sd.getCurrentDir(), layoutVersion, cid);
if(cid != null && !cid.equals("")) {
return cid;
}
} catch (Exception e) {
LOG.warn("this sd not available: " + e.getLocalizedMessage());
LOG.warn("this sd not available: {}", e.getLocalizedMessage());
} //ignore
}
LOG.warn("couldn't find any VERSION file containing valid ClusterId");
@ -1151,8 +1156,8 @@ public class NNStorage extends Storage implements Closeable,
try {
writeProperties(sd);
} catch (Exception e) {
LOG.warn("Error during write properties to the VERSION file to " +
sd.toString(), e);
LOG.warn("Error during write properties to the VERSION file to {}",
sd, e);
reportErrorsOnDirectory(sd);
if (getStorageDirs().isEmpty()) {
throw new IOException("All the storage failed while writing " +

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.junit.Test;
import org.mockito.Mockito;
@ -26,6 +24,8 @@ import org.mockito.Mockito;
import java.io.File;
import java.util.Random;
import java.util.UUID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertThat;
@ -35,7 +35,9 @@ import static org.junit.Assert.assertThat;
* restore directories for a given block file path.
*/
public class TestBlockPoolSliceStorage {
public static final Log LOG = LogFactory.getLog(TestBlockPoolSliceStorage.class);
public static final Logger LOG = LoggerFactory
.getLogger(TestBlockPoolSliceStorage.class);
final Random rand = new Random();
BlockPoolSliceStorage storage;
@ -104,8 +106,8 @@ public class TestBlockPoolSliceStorage {
BlockPoolSliceStorage.TRASH_ROOT_DIR +
blockFileSubdir.substring(0, blockFileSubdir.length() - 1);
LOG.info("Got subdir " + blockFileSubdir);
LOG.info("Generated file path " + testFilePath);
LOG.info("Got subdir {}", blockFileSubdir);
LOG.info("Generated file path {}", testFilePath);
ReplicaInfo info = Mockito.mock(ReplicaInfo.class);
Mockito.when(info.getBlockURI()).thenReturn(new File(testFilePath).toURI());
@ -131,7 +133,7 @@ public class TestBlockPoolSliceStorage {
Storage.STORAGE_DIR_CURRENT +
blockFileSubdir.substring(0, blockFileSubdir.length() - 1);
LOG.info("Generated deleted file path " + deletedFilePath);
LOG.info("Generated deleted file path {}", deletedFilePath);
assertThat(storage.getRestoreDirectory(new File(deletedFilePath)),
is(expectedRestorePath));