|
@@ -69,9 +69,9 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
static final String DEPRECATED_MESSAGE_DIGEST_PROPERTY = "imageMD5Digest";
|
|
|
static final String LOCAL_URI_SCHEME = "file";
|
|
|
|
|
|
- //
|
|
|
- // The filenames used for storing the images
|
|
|
- //
|
|
|
+ /**
|
|
|
+ * The filenames used for storing the images.
|
|
|
+ */
|
|
|
public enum NameNodeFile {
|
|
|
IMAGE ("fsimage"),
|
|
|
TIME ("fstime"), // from "old" pre-HDFS-1073 format
|
|
@@ -85,9 +85,14 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
IMAGE_LEGACY_OIV ("fsimage_legacy_oiv"); // For pre-PB format
|
|
|
|
|
|
private String fileName = null;
|
|
|
- private NameNodeFile(String name) { this.fileName = name; }
|
|
|
+ NameNodeFile(String name) {
|
|
|
+ this.fileName = name;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
- public String getName() { return fileName; }
|
|
|
+ public String getName() {
|
|
|
+ return fileName;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -97,7 +102,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* stores both fsimage and edits.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- public static enum NameNodeDirType implements StorageDirType {
|
|
|
+ public enum NameNodeDirType implements StorageDirType {
|
|
|
UNDEFINED,
|
|
|
IMAGE,
|
|
|
EDITS,
|
|
@@ -110,16 +115,15 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
|
|
|
@Override
|
|
|
public boolean isOfType(StorageDirType type) {
|
|
|
- if ((this == IMAGE_AND_EDITS) && (type == IMAGE || type == EDITS))
|
|
|
- return true;
|
|
|
- return this == type;
|
|
|
+ return (this == IMAGE_AND_EDITS) && (type == IMAGE || type == EDITS) ||
|
|
|
+ this == type;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
protected String blockpoolID = ""; // id of the block pool
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- * flag that controls if we try to restore failed storages
|
|
|
+ * Flag that controls if we try to restore failed storages.
|
|
|
*/
|
|
|
private boolean restoreFailedStorage = false;
|
|
|
private final Object restorationLock = new Object();
|
|
@@ -131,7 +135,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* recent fsimage file. This does not include any transactions
|
|
|
* that have since been written to the edit log.
|
|
|
*/
|
|
|
- protected volatile long mostRecentCheckpointTxId = HdfsServerConstants.INVALID_TXID;
|
|
|
+ protected volatile long mostRecentCheckpointTxId =
|
|
|
+ HdfsServerConstants.INVALID_TXID;
|
|
|
|
|
|
/**
|
|
|
* Time of the last checkpoint, in milliseconds since the epoch.
|
|
@@ -139,10 +144,10 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
private long mostRecentCheckpointTime = 0;
|
|
|
|
|
|
/**
|
|
|
- * list of failed (and thus removed) storages
|
|
|
+ * List of failed (and thus removed) storages.
|
|
|
*/
|
|
|
final protected List<StorageDirectory> removedStorageDirs
|
|
|
- = new CopyOnWriteArrayList<StorageDirectory>();
|
|
|
+ = new CopyOnWriteArrayList<>();
|
|
|
|
|
|
/**
|
|
|
* Properties from old layout versions that may be needed
|
|
@@ -167,7 +172,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
throws IOException {
|
|
|
super(NodeType.NAME_NODE);
|
|
|
|
|
|
- storageDirs = new CopyOnWriteArrayList<StorageDirectory>();
|
|
|
+ storageDirs = new CopyOnWriteArrayList<>();
|
|
|
|
|
|
// this may modify the editsDirs, so copy before passing in
|
|
|
setStorageDirectories(imageDirs,
|
|
@@ -195,8 +200,9 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
int oldVersion = oldFile.readInt();
|
|
|
oldFile.close();
|
|
|
oldFile = null;
|
|
|
- if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
|
|
|
+ if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION) {
|
|
|
return false;
|
|
|
+ }
|
|
|
} finally {
|
|
|
IOUtils.cleanup(LOG, oldFile);
|
|
|
}
|
|
@@ -233,21 +239,19 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*/
|
|
|
void attemptRestoreRemovedStorage() {
|
|
|
// if directory is "alive" - copy the images there...
|
|
|
- if(!restoreFailedStorage || removedStorageDirs.size() == 0)
|
|
|
+ if(!restoreFailedStorage || removedStorageDirs.size() == 0) {
|
|
|
return; //nothing to restore
|
|
|
-
|
|
|
+ }
|
|
|
/* We don't want more than one thread trying to restore at a time */
|
|
|
synchronized (this.restorationLock) {
|
|
|
LOG.info("NNStorage.attemptRestoreRemovedStorage: check removed(failed) "+
|
|
|
"storarge. removedStorages size = " + removedStorageDirs.size());
|
|
|
- for(Iterator<StorageDirectory> it
|
|
|
- = this.removedStorageDirs.iterator(); it.hasNext();) {
|
|
|
- StorageDirectory sd = it.next();
|
|
|
+ for (StorageDirectory sd : this.removedStorageDirs) {
|
|
|
File root = sd.getRoot();
|
|
|
LOG.info("currently disabled dir " + root.getAbsolutePath() +
|
|
|
- "; type="+sd.getStorageDirType()
|
|
|
- + ";canwrite="+FileUtil.canWrite(root));
|
|
|
- if(root.exists() && FileUtil.canWrite(root)) {
|
|
|
+ "; type=" + sd.getStorageDirType()
|
|
|
+ + ";canwrite=" + FileUtil.canWrite(root));
|
|
|
+ if (root.exists() && FileUtil.canWrite(root)) {
|
|
|
LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
|
|
|
this.addStorageDir(sd); // restore
|
|
|
this.removedStorageDirs.remove(sd);
|
|
@@ -264,13 +268,13 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * See {@link NNStorage#setStorageDirectories(Collection, Collection, Collection)}
|
|
|
+ * See {@link NNStorage#setStorageDirectories(Collection, Collection, Collection)}.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
synchronized void setStorageDirectories(Collection<URI> fsNameDirs,
|
|
|
Collection<URI> fsEditsDirs)
|
|
|
throws IOException {
|
|
|
- setStorageDirectories(fsNameDirs, fsEditsDirs, new ArrayList<URI>());
|
|
|
+ setStorageDirectories(fsNameDirs, fsEditsDirs, new ArrayList<>());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -322,14 +326,15 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
checkSchemeConsistency(dirName);
|
|
|
// Add to the list of storage directories, only if the
|
|
|
// URI is of type file://
|
|
|
- if(dirName.getScheme().compareTo("file") == 0)
|
|
|
+ if(dirName.getScheme().compareTo("file") == 0) {
|
|
|
this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
|
|
|
- NameNodeDirType.EDITS, sharedEditsDirs.contains(dirName)));
|
|
|
+ NameNodeDirType.EDITS, sharedEditsDirs.contains(dirName)));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Return the storage directory corresponding to the passed URI
|
|
|
+ * Return the storage directory corresponding to the passed URI.
|
|
|
* @param uri URI of a storage directory
|
|
|
* @return The matching storage directory or null if none found
|
|
|
*/
|
|
@@ -337,7 +342,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
try {
|
|
|
uri = Util.fileAsURI(new File(uri));
|
|
|
Iterator<StorageDirectory> it = dirIterator();
|
|
|
- for (; it.hasNext(); ) {
|
|
|
+ while (it.hasNext()) {
|
|
|
StorageDirectory sd = it.next();
|
|
|
if (Util.fileAsURI(sd.getRoot()).equals(uri)) {
|
|
|
return sd;
|
|
@@ -351,7 +356,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
|
|
|
/**
|
|
|
* Checks the consistency of a URI, in particular if the scheme
|
|
|
- * is specified
|
|
|
+ * is specified.
|
|
|
* @param u URI whose consistency is being checked.
|
|
|
*/
|
|
|
private static void checkSchemeConsistency(URI u) throws IOException {
|
|
@@ -363,7 +368,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Retrieve current directories of type IMAGE
|
|
|
+ * Retrieve current directories of type IMAGE.
|
|
|
* @return Collection of URI representing image directories
|
|
|
* @throws IOException in case of URI processing error
|
|
|
*/
|
|
@@ -372,7 +377,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Retrieve current directories of type EDITS
|
|
|
+ * Retrieve current directories of type EDITS.
|
|
|
* @return Collection of URI representing edits directories
|
|
|
* @throws IOException in case of URI processing error
|
|
|
*/
|
|
@@ -386,12 +391,14 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* @return number of storage directories of type dirType
|
|
|
*/
|
|
|
int getNumStorageDirs(NameNodeDirType dirType) {
|
|
|
- if(dirType == null)
|
|
|
+ if(dirType == null) {
|
|
|
return getNumStorageDirs();
|
|
|
+ }
|
|
|
Iterator<StorageDirectory> it = dirIterator(dirType);
|
|
|
int numDirs = 0;
|
|
|
- for(; it.hasNext(); it.next())
|
|
|
+ for(; it.hasNext(); it.next()) {
|
|
|
numDirs++;
|
|
|
+ }
|
|
|
return numDirs;
|
|
|
}
|
|
|
|
|
@@ -404,10 +411,10 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*/
|
|
|
Collection<URI> getDirectories(NameNodeDirType dirType)
|
|
|
throws IOException {
|
|
|
- ArrayList<URI> list = new ArrayList<URI>();
|
|
|
+ ArrayList<URI> list = new ArrayList<>();
|
|
|
Iterator<StorageDirectory> it = (dirType == null) ? dirIterator() :
|
|
|
dirIterator(dirType);
|
|
|
- for ( ;it.hasNext(); ) {
|
|
|
+ for ( ; it.hasNext();) {
|
|
|
StorageDirectory sd = it.next();
|
|
|
try {
|
|
|
list.add(Util.fileAsURI(sd.getRoot()));
|
|
@@ -440,7 +447,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* @param sd storage directory
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void writeTransactionIdFile(StorageDirectory sd, long txid) throws IOException {
|
|
|
+ void writeTransactionIdFile(StorageDirectory sd, long txid)
|
|
|
+ throws IOException {
|
|
|
Preconditions.checkArgument(txid >= 0, "bad txid: " + txid);
|
|
|
|
|
|
File txIdFile = getStorageFile(sd, NameNodeFile.SEEN_TXID);
|
|
@@ -448,7 +456,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Set the transaction ID and time of the last checkpoint
|
|
|
+ * Set the transaction ID and time of the last checkpoint.
|
|
|
*
|
|
|
* @param txid transaction id of the last checkpoint
|
|
|
* @param time time of the last checkpoint, in millis since the epoch
|
|
@@ -513,12 +521,12 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
|
|
|
/**
|
|
|
* Return the name of the image file that is uploaded by periodic
|
|
|
- * checkpointing
|
|
|
+ * checkpointing.
|
|
|
*
|
|
|
* @return List of filenames to save checkpoints to.
|
|
|
*/
|
|
|
public File[] getFsImageNameCheckpoint(long txid) {
|
|
|
- ArrayList<File> list = new ArrayList<File>();
|
|
|
+ ArrayList<File> list = new ArrayList<>();
|
|
|
for (Iterator<StorageDirectory> it =
|
|
|
dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
|
|
|
list.add(getStorageFile(it.next(), NameNodeFile.IMAGE_NEW, txid));
|
|
@@ -583,7 +591,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*/
|
|
|
public void format(NamespaceInfo nsInfo) throws IOException {
|
|
|
Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
|
|
|
- nsInfo.getLayoutVersion() == HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
|
|
|
+ nsInfo.getLayoutVersion() ==
|
|
|
+ HdfsServerConstants.NAMENODE_LAYOUT_VERSION,
|
|
|
"Bad layout version: %s", nsInfo.getLayoutVersion());
|
|
|
|
|
|
this.setStorageInfo(nsInfo);
|
|
@@ -624,8 +633,9 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*/
|
|
|
private static int newNamespaceID() {
|
|
|
int newID = 0;
|
|
|
- while(newID == 0)
|
|
|
+ while(newID == 0) {
|
|
|
newID = ThreadLocalRandom.current().nextInt(0x7FFFFFFF); // use 31 bits
|
|
|
+ }
|
|
|
return newID;
|
|
|
}
|
|
|
|
|
@@ -650,8 +660,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
void readProperties(StorageDirectory sd, StartupOption startupOption)
|
|
|
throws IOException {
|
|
|
Properties props = readPropertiesFile(sd.getVersionFile());
|
|
|
- if (HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK.matches
|
|
|
- (startupOption)) {
|
|
|
+ if (HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK
|
|
|
+ .matches(startupOption)) {
|
|
|
int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
|
|
|
if (lv > getServiceLayoutVersion()) {
|
|
|
// we should not use a newer version for rollingUpgrade rollback
|
|
@@ -669,7 +679,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* versions of HDFS and only necessary during upgrade.
|
|
|
*/
|
|
|
private void setDeprecatedPropertiesForUpgrade(Properties props) {
|
|
|
- deprecatedProperties = new HashMap<String, String>();
|
|
|
+ deprecatedProperties = new HashMap<>();
|
|
|
String md5 = props.getProperty(DEPRECATED_MESSAGE_DIGEST_PROPERTY);
|
|
|
if (md5 != null) {
|
|
|
deprecatedProperties.put(DEPRECATED_MESSAGE_DIGEST_PROPERTY, md5);
|
|
@@ -700,8 +710,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*/
|
|
|
@Override // Storage
|
|
|
protected void setPropertiesFromFields(Properties props,
|
|
|
- StorageDirectory sd
|
|
|
- ) throws IOException {
|
|
|
+ StorageDirectory sd) throws IOException {
|
|
|
super.setPropertiesFromFields(props, sd);
|
|
|
// Set blockpoolID in version with federation support
|
|
|
if (NameNodeLayoutVersion.supports(
|
|
@@ -710,14 +719,15 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static File getStorageFile(StorageDirectory sd, NameNodeFile type, long imageTxId) {
|
|
|
+ static File getStorageFile(StorageDirectory sd, NameNodeFile type,
|
|
|
+ long imageTxId) {
|
|
|
return new File(sd.getCurrentDir(),
|
|
|
String.format("%s_%019d", type.getName(), imageTxId));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Get a storage file for one of the files that doesn't need a txid associated
|
|
|
- * (e.g version, seen_txid)
|
|
|
+ * (e.g version, seen_txid).
|
|
|
*/
|
|
|
static File getStorageFile(StorageDirectory sd, NameNodeFile type) {
|
|
|
return new File(sd.getCurrentDir(), type.getName());
|
|
@@ -779,8 +789,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
|
|
|
public static String getTemporaryEditsFileName(long startTxId, long endTxId,
|
|
|
long timestamp) {
|
|
|
- return String.format("%s_%019d-%019d_%019d", NameNodeFile.EDITS_TMP.getName(),
|
|
|
- startTxId, endTxId, timestamp);
|
|
|
+ return String.format("%s_%019d-%019d_%019d",
|
|
|
+ NameNodeFile.EDITS_TMP.getName(), startTxId, endTxId, timestamp);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -799,7 +809,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
|
|
|
/**
|
|
|
* Return the first readable image file for the given txid and image type, or
|
|
|
- * null if no such image can be found
|
|
|
+ * null if no such image can be found.
|
|
|
*/
|
|
|
File findImageFile(NameNodeFile nnf, long txid) {
|
|
|
return findFile(NameNodeDirType.IMAGE,
|
|
@@ -878,9 +888,10 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* @param layoutVersion Layout version for the upgrade
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
|
|
|
- throws IOException {
|
|
|
- if (startOpt == StartupOption.UPGRADE || startOpt == StartupOption.UPGRADEONLY) {
|
|
|
+ void processStartupOptionsForUpgrade(StartupOption startOpt,
|
|
|
+ int layoutVersion) throws IOException {
|
|
|
+ if (startOpt == StartupOption.UPGRADE ||
|
|
|
+ startOpt == StartupOption.UPGRADEONLY) {
|
|
|
// If upgrade from a release that does not support federation,
|
|
|
// if clusterId is provided in the startupOptions use it.
|
|
|
// Else generate a new cluster ID
|
|
@@ -935,10 +946,10 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
*
|
|
|
* clusterID is a persistent attribute of the cluster.
|
|
|
* It is generated when the cluster is created and remains the same
|
|
|
- * during the life cycle of the cluster. When a new name node is formated, if
|
|
|
- * this is a new cluster, a new clusterID is geneated and stored. Subsequent
|
|
|
- * name node must be given the same ClusterID during its format to be in the
|
|
|
- * same cluster.
|
|
|
+ * during the life cycle of the cluster. When a new name node is formated,
|
|
|
+ * if this is a new cluster, a new clusterID is geneated and stored.
|
|
|
+ * Subsequent name node must be given the same ClusterID during its format to
|
|
|
+ * be in the same cluster.
|
|
|
* When a datanode register it receive the clusterID and stick with it.
|
|
|
* If at any point, name node or data node tries to join another cluster, it
|
|
|
* will be rejected.
|
|
@@ -954,13 +965,13 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * try to find current cluster id in the VERSION files
|
|
|
+ * Try to find current cluster id in the VERSION files.
|
|
|
* returns first cluster id found in any VERSION file
|
|
|
* null in case none found
|
|
|
* @return clusterId or null in case no cluster id found
|
|
|
*/
|
|
|
public String determineClusterId() {
|
|
|
- String cid = null;
|
|
|
+ String cid;
|
|
|
Iterator<StorageDirectory> sdit = dirIterator(NameNodeDirType.IMAGE);
|
|
|
while(sdit.hasNext()) {
|
|
|
StorageDirectory sd = sdit.next();
|
|
@@ -970,8 +981,9 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
LOG.info("current cluster id for sd="+sd.getCurrentDir() +
|
|
|
";lv=" + layoutVersion + ";cid=" + cid);
|
|
|
|
|
|
- if(cid != null && !cid.equals(""))
|
|
|
+ if(cid != null && !cid.equals("")) {
|
|
|
return cid;
|
|
|
+ }
|
|
|
} catch (Exception e) {
|
|
|
LOG.warn("this sd not available: " + e.getLocalizedMessage());
|
|
|
} //ignore
|
|
@@ -986,7 +998,7 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* @return new blockpoolID
|
|
|
*/
|
|
|
static String newBlockPoolID() throws UnknownHostException{
|
|
|
- String ip = "unknownIP";
|
|
|
+ String ip;
|
|
|
try {
|
|
|
ip = DNS.getDefaultIP("default");
|
|
|
} catch (UnknownHostException e) {
|
|
@@ -995,16 +1007,15 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
}
|
|
|
|
|
|
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
|
|
|
- String bpid = "BP-" + rand + "-"+ ip + "-" + Time.now();
|
|
|
- return bpid;
|
|
|
+ return "BP-" + rand + "-"+ ip + "-" + Time.now();
|
|
|
}
|
|
|
|
|
|
- /** Validate and set block pool ID */
|
|
|
+ /** Validate and set block pool ID. */
|
|
|
public void setBlockPoolID(String bpid) {
|
|
|
blockpoolID = bpid;
|
|
|
}
|
|
|
|
|
|
- /** Validate and set block pool ID */
|
|
|
+ /** Validate and set block pool ID. */
|
|
|
private void setBlockPoolID(File storage, String bpid)
|
|
|
throws InconsistentFSStateException {
|
|
|
if (bpid == null || bpid.equals("")) {
|
|
@@ -1044,7 +1055,8 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
* inspected each directory.
|
|
|
*
|
|
|
* <b>Note:</b> this can mutate the storage info fields (ctime, version, etc).
|
|
|
- * @throws IOException if no valid storage dirs are found or no valid layout version
|
|
|
+ * @throws IOException if no valid storage dirs are found or no valid layout
|
|
|
+ * version
|
|
|
*/
|
|
|
FSImageStorageInspector readAndInspectDirs(EnumSet<NameNodeFile> fileTypes,
|
|
|
StartupOption startupOption) throws IOException {
|
|
@@ -1057,21 +1069,24 @@ public class NNStorage extends Storage implements Closeable,
|
|
|
it.hasNext();) {
|
|
|
StorageDirectory sd = it.next();
|
|
|
if (!sd.getVersionFile().exists()) {
|
|
|
- FSImage.LOG.warn("Storage directory " + sd + " contains no VERSION file. Skipping...");
|
|
|
+ FSImage.LOG.warn("Storage directory " + sd +
|
|
|
+ " contains no VERSION file. Skipping...");
|
|
|
continue;
|
|
|
}
|
|
|
readProperties(sd, startupOption); // sets layoutVersion
|
|
|
int lv = getLayoutVersion();
|
|
|
if (layoutVersion == null) {
|
|
|
- layoutVersion = Integer.valueOf(lv);
|
|
|
+ layoutVersion = lv;
|
|
|
} else if (!layoutVersion.equals(lv)) {
|
|
|
multipleLV = true;
|
|
|
}
|
|
|
- layoutVersions.append("(").append(sd.getRoot()).append(", ").append(lv).append(") ");
|
|
|
+ layoutVersions.append("(").append(sd.getRoot()).append(", ").append(lv)
|
|
|
+ .append(") ");
|
|
|
}
|
|
|
|
|
|
if (layoutVersion == null) {
|
|
|
- throw new IOException("No storage directories contained VERSION information");
|
|
|
+ throw new IOException("No storage directories contained VERSION" +
|
|
|
+ " information");
|
|
|
}
|
|
|
if (multipleLV) {
|
|
|
throw new IOException(
|