|
@@ -96,9 +96,8 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
FSDirectory dir;
|
|
FSDirectory dir;
|
|
|
|
|
|
//
|
|
//
|
|
- // Stores the block-->datanode(s) map. Updated only in response
|
|
|
|
- // to client-sent information.
|
|
|
|
// Mapping: Block -> { INode, datanodes, self ref }
|
|
// Mapping: Block -> { INode, datanodes, self ref }
|
|
|
|
+ // Updated only in response to client-sent information.
|
|
//
|
|
//
|
|
BlocksMap blocksMap = new BlocksMap();
|
|
BlocksMap blocksMap = new BlocksMap();
|
|
|
|
|
|
@@ -238,9 +237,8 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
|
|
|
|
/**
|
|
/**
|
|
* The global generation stamp for this file system.
|
|
* The global generation stamp for this file system.
|
|
- * Valid values start from 1000.
|
|
|
|
*/
|
|
*/
|
|
- private GenerationStamp generationStamp = new GenerationStamp(1000);
|
|
|
|
|
|
+ private final GenerationStamp generationStamp = new GenerationStamp();
|
|
|
|
|
|
// Ask Datanode only up to this many blocks to delete.
|
|
// Ask Datanode only up to this many blocks to delete.
|
|
private int blockInvalidateLimit = FSConstants.BLOCK_INVALIDATE_CHUNK;
|
|
private int blockInvalidateLimit = FSConstants.BLOCK_INVALIDATE_CHUNK;
|
|
@@ -931,7 +929,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
// If the file is under construction , then it must be in our
|
|
// If the file is under construction , then it must be in our
|
|
// leases. Find the appropriate lease record.
|
|
// leases. Find the appropriate lease record.
|
|
//
|
|
//
|
|
- Lease lease = leaseManager.getLease(holder);
|
|
|
|
|
|
+ Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
|
|
//
|
|
//
|
|
// We found the lease for this file. And surprisingly the original
|
|
// We found the lease for this file. And surprisingly the original
|
|
// holder is trying to recreate this file. This should never occur.
|
|
// holder is trying to recreate this file. This should never occur.
|
|
@@ -945,7 +943,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
//
|
|
//
|
|
// Find the original holder.
|
|
// Find the original holder.
|
|
//
|
|
//
|
|
- lease = leaseManager.getLease(pendingFile.getClientName());
|
|
|
|
|
|
+ lease = leaseManager.getLease(pendingFile.clientName);
|
|
if (lease == null) {
|
|
if (lease == null) {
|
|
throw new AlreadyBeingCreatedException(
|
|
throw new AlreadyBeingCreatedException(
|
|
"failed to create file " + src + " for " + holder +
|
|
"failed to create file " + src + " for " + holder +
|
|
@@ -958,8 +956,9 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
// to proceed. Otherwise, prevent this request from creating file.
|
|
// to proceed. Otherwise, prevent this request from creating file.
|
|
//
|
|
//
|
|
if (lease.expiredSoftLimit()) {
|
|
if (lease.expiredSoftLimit()) {
|
|
- LOG.info("startFile: Removing lease " + lease);
|
|
|
|
- leaseManager.removeExpiredLease(lease);
|
|
|
|
|
|
+ LOG.info("startFile: recover lease " + lease + ", src=" + src);
|
|
|
|
+ internalReleaseLease(lease, src);
|
|
|
|
+ leaseManager.renewLease(lease);
|
|
} else {
|
|
} else {
|
|
throw new AlreadyBeingCreatedException(
|
|
throw new AlreadyBeingCreatedException(
|
|
"failed to create file " + src + " for " + holder +
|
|
"failed to create file " + src + " for " + holder +
|
|
@@ -988,8 +987,6 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
DatanodeDescriptor clientNode =
|
|
DatanodeDescriptor clientNode =
|
|
host2DataNodeMap.getDatanodeByHost(clientMachine);
|
|
host2DataNodeMap.getDatanodeByHost(clientMachine);
|
|
|
|
|
|
- leaseManager.addLease(src, holder);
|
|
|
|
-
|
|
|
|
//
|
|
//
|
|
// Now we can add the name to the filesystem. This file has no
|
|
// Now we can add the name to the filesystem. This file has no
|
|
// blocks associated with it.
|
|
// blocks associated with it.
|
|
@@ -997,22 +994,64 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
checkFsObjectLimit();
|
|
checkFsObjectLimit();
|
|
|
|
|
|
// increment global generation stamp
|
|
// increment global generation stamp
|
|
- long genstamp = generationStamp.nextStamp();
|
|
|
|
-
|
|
|
|
- INode newNode = dir.addFile(src, permissions,
|
|
|
|
|
|
+ long genstamp = nextGenerationStamp();
|
|
|
|
+ INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
|
|
replication, blockSize, holder, clientMachine, clientNode, genstamp);
|
|
replication, blockSize, holder, clientMachine, clientNode, genstamp);
|
|
if (newNode == null) {
|
|
if (newNode == null) {
|
|
throw new IOException("DIR* NameSystem.startFile: " +
|
|
throw new IOException("DIR* NameSystem.startFile: " +
|
|
"Unable to add file to namespace.");
|
|
"Unable to add file to namespace.");
|
|
}
|
|
}
|
|
|
|
+ leaseManager.addLease(newNode.clientName, src);
|
|
} catch (IOException ie) {
|
|
} catch (IOException ie) {
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
|
|
+ie.getMessage());
|
|
+ie.getMessage());
|
|
throw ie;
|
|
throw ie;
|
|
}
|
|
}
|
|
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
|
|
|
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
|
|
+"add "+src+" to namespace for "+holder);
|
|
+"add "+src+" to namespace for "+holder);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** append is not yet ready. This method is for testing. */
|
|
|
|
+ void appendFileInternal(String src, String holder, String clientMachine
|
|
|
|
+ ) throws IOException {
|
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: file "
|
|
|
|
+ +src+" for "+holder+" at "+clientMachine);
|
|
|
|
+ }
|
|
|
|
+ if (isInSafeMode())
|
|
|
|
+ throw new SafeModeException("Cannot append file" + src, safeMode);
|
|
|
|
+ if (!isValidName(src)) {
|
|
|
|
+ throw new IOException("Invalid file name: " + src);
|
|
|
|
+ }
|
|
|
|
+ if (isPermissionEnabled) {
|
|
|
|
+ checkPathAccess(src, FsAction.WRITE);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ INodeFile f = dir.getFileINode(src);
|
|
|
|
+ //assume f != null && !f.isUnderConstruction() && lease does not exist
|
|
|
|
+ //TODO: remove the assumption
|
|
|
|
+
|
|
|
|
+ DatanodeDescriptor clientNode = host2DataNodeMap.getDatanodeByHost(
|
|
|
|
+ clientMachine);
|
|
|
|
+ INodeFileUnderConstruction newnode = f.toINodeFileUnderConstruction(
|
|
|
|
+ holder, clientMachine, clientNode);
|
|
|
|
+
|
|
|
|
+ dir.replaceNode(src, f, newnode);
|
|
|
|
+ leaseManager.addLease(newnode.clientName, src);
|
|
|
|
+
|
|
|
|
+ } catch (IOException ie) {
|
|
|
|
+ NameNode.stateChangeLog.warn("DIR* NameSystem.appendFile: ", ie);
|
|
|
|
+ throw ie;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: "
|
|
|
|
+ +"add "+src+" to namespace for "+holder);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1079,6 +1118,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
|
|
|
|
// allocate new block record block locations in INode.
|
|
// allocate new block record block locations in INode.
|
|
newBlock = allocateBlock(src, pendingFile);
|
|
newBlock = allocateBlock(src, pendingFile);
|
|
|
|
+ pendingFile.targets = targets;
|
|
}
|
|
}
|
|
|
|
|
|
// Create next block
|
|
// Create next block
|
|
@@ -1108,7 +1148,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
) throws IOException {
|
|
) throws IOException {
|
|
INode file = dir.getFileINode(src);
|
|
INode file = dir.getFileINode(src);
|
|
if (file == null) {
|
|
if (file == null) {
|
|
- Lease lease = leaseManager.getLease(holder);
|
|
|
|
|
|
+ Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
|
|
throw new LeaseExpiredException("No lease on " + src +
|
|
throw new LeaseExpiredException("No lease on " + src +
|
|
" File does not exist. " +
|
|
" File does not exist. " +
|
|
(lease != null ? lease.toString() :
|
|
(lease != null ? lease.toString() :
|
|
@@ -1116,7 +1156,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
" does not have any open files."));
|
|
" does not have any open files."));
|
|
}
|
|
}
|
|
if (!file.isUnderConstruction()) {
|
|
if (!file.isUnderConstruction()) {
|
|
- Lease lease = leaseManager.getLease(holder);
|
|
|
|
|
|
+ Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
|
|
throw new LeaseExpiredException("No lease on " + src +
|
|
throw new LeaseExpiredException("No lease on " + src +
|
|
" File is not open for writing. " +
|
|
" File is not open for writing. " +
|
|
(lease != null ? lease.toString() :
|
|
(lease != null ? lease.toString() :
|
|
@@ -1167,30 +1207,13 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
} else if (!checkFileProgress(pendingFile, true)) {
|
|
} else if (!checkFileProgress(pendingFile, true)) {
|
|
return STILL_WAITING;
|
|
return STILL_WAITING;
|
|
}
|
|
}
|
|
-
|
|
|
|
- // The file is no longer pending.
|
|
|
|
- // Create permanent INode, update blockmap
|
|
|
|
- INodeFile newFile = pendingFile.convertToInodeFile();
|
|
|
|
- dir.replaceNode(src, pendingFile, newFile);
|
|
|
|
|
|
|
|
- // close file and persist block allocations for this file
|
|
|
|
- dir.closeFile(src, newFile);
|
|
|
|
|
|
+ finalizeINodeFileUnderConstruction(src, pendingFile);
|
|
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src
|
|
|
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src
|
|
+ " blocklist persisted");
|
|
+ " blocklist persisted");
|
|
-
|
|
|
|
- leaseManager.removeLease(src, holder);
|
|
|
|
-
|
|
|
|
- //
|
|
|
|
- // REMIND - mjc - this should be done only after we wait a few secs.
|
|
|
|
- // The namenode isn't giving datanodes enough time to report the
|
|
|
|
- // replicated blocks that are automatically done as part of a client
|
|
|
|
- // write.
|
|
|
|
- //
|
|
|
|
-
|
|
|
|
- // Now that the file is real, we need to be sure to replicate
|
|
|
|
- // the blocks.
|
|
|
|
- checkReplicationFactor(newFile);
|
|
|
|
|
|
+ }
|
|
return COMPLETE_SUCCESS;
|
|
return COMPLETE_SUCCESS;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1291,27 +1314,15 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
* dumps the contents of recentInvalidateSets
|
|
* dumps the contents of recentInvalidateSets
|
|
*/
|
|
*/
|
|
private synchronized void dumpRecentInvalidateSets(PrintWriter out) {
|
|
private synchronized void dumpRecentInvalidateSets(PrintWriter out) {
|
|
- Collection<Collection<Block>> values = recentInvalidateSets.values();
|
|
|
|
- Iterator<Map.Entry<String,Collection<Block>>> it =
|
|
|
|
- recentInvalidateSets.entrySet().iterator();
|
|
|
|
- if (values.size() == 0) {
|
|
|
|
- out.println("Metasave: Blocks waiting deletion: 0");
|
|
|
|
|
|
+ int size = recentInvalidateSets.values().size();
|
|
|
|
+ out.println("Metasave: Blocks waiting deletion from "+size+" datanodes.");
|
|
|
|
+ if (size == 0) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- out.println("Metasave: Blocks waiting deletion from " +
|
|
|
|
- values.size() + " datanodes.");
|
|
|
|
- while (it.hasNext()) {
|
|
|
|
- Map.Entry<String,Collection<Block>> entry = it.next();
|
|
|
|
- String storageId = entry.getKey();
|
|
|
|
- DatanodeDescriptor node = datanodeMap.get(storageId);
|
|
|
|
- Collection<Block> blklist = entry.getValue();
|
|
|
|
- if (blklist.size() > 0) {
|
|
|
|
- out.print(node.getName());
|
|
|
|
- for (Iterator jt = blklist.iterator(); jt.hasNext();) {
|
|
|
|
- Block block = (Block) jt.next();
|
|
|
|
- out.print(" " + block);
|
|
|
|
- }
|
|
|
|
- out.println("");
|
|
|
|
|
|
+ for(Map.Entry<String,Collection<Block>> entry : recentInvalidateSets.entrySet()) {
|
|
|
|
+ Collection<Block> blocks = entry.getValue();
|
|
|
|
+ if (blocks.size() > 0) {
|
|
|
|
+ out.println(datanodeMap.get(entry.getKey()).getName() + blocks);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1478,7 +1489,7 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
}
|
|
}
|
|
if (old.isUnderConstruction()) {
|
|
if (old.isUnderConstruction()) {
|
|
INodeFileUnderConstruction cons = (INodeFileUnderConstruction) old;
|
|
INodeFileUnderConstruction cons = (INodeFileUnderConstruction) old;
|
|
- leaseManager.removeLease(src, cons.getClientName());
|
|
|
|
|
|
+ leaseManager.removeLease(cons.clientName, src);
|
|
}
|
|
}
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
@@ -1602,7 +1613,11 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
* @param src The filename
|
|
* @param src The filename
|
|
* @param holder The datanode that was creating the file
|
|
* @param holder The datanode that was creating the file
|
|
*/
|
|
*/
|
|
- void internalReleaseCreate(String src, String holder) throws IOException {
|
|
|
|
|
|
+ void internalReleaseLease(Lease lease, String src) throws IOException {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("lease=" + lease + ", src=" + src);
|
|
|
|
+ }
|
|
|
|
+
|
|
INodeFile iFile = dir.getFileINode(src);
|
|
INodeFile iFile = dir.getFileINode(src);
|
|
if (iFile == null) {
|
|
if (iFile == null) {
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.internalReleaseCreate: "
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.internalReleaseCreate: "
|
|
@@ -1616,30 +1631,20 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
+ src + " but file is already closed.");
|
|
+ src + " but file is already closed.");
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.internalReleaseCreate: "
|
|
|
|
+ + src + " does not being written in " + lease);
|
|
|
|
+ }
|
|
|
|
+
|
|
INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) iFile;
|
|
INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) iFile;
|
|
|
|
|
|
- // The last block that was allocated migth not have been used by the
|
|
|
|
- // client. In this case, the size of the last block would be 0. A fsck
|
|
|
|
- // will report this block as a missing block because no datanodes have it.
|
|
|
|
- // Delete this block.
|
|
|
|
- Block[] blocks = pendingFile.getBlocks();
|
|
|
|
- if (blocks != null && blocks.length > 0) {
|
|
|
|
- Block last = blocks[blocks.length - 1];
|
|
|
|
- if (last.getNumBytes() == 0) {
|
|
|
|
- pendingFile.removeBlock(last);
|
|
|
|
- blocksMap.removeINode(last);
|
|
|
|
- for (Iterator<DatanodeDescriptor> it =
|
|
|
|
- blocksMap.nodeIterator(last); it.hasNext();) {
|
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
|
- addToInvalidates(last, node);
|
|
|
|
- }
|
|
|
|
- /* What else do we need to do?
|
|
|
|
- * removeStoredBlock()? we do different things when a block is
|
|
|
|
- * removed in different contexts. Mostly these should be
|
|
|
|
- * same and/or should be in one place.
|
|
|
|
- */
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ // Initialize lease recovery for pendingFile
|
|
|
|
+ pendingFile.assignPrimaryDatanode();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void finalizeINodeFileUnderConstruction(String src,
|
|
|
|
+ INodeFileUnderConstruction pendingFile) throws IOException {
|
|
|
|
+ leaseManager.removeLease(pendingFile.clientName, src);
|
|
|
|
|
|
// The file is no longer pending.
|
|
// The file is no longer pending.
|
|
// Create permanent INode, update blockmap
|
|
// Create permanent INode, update blockmap
|
|
@@ -1649,14 +1654,58 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
// close file and persist block allocations for this file
|
|
// close file and persist block allocations for this file
|
|
dir.closeFile(src, newFile);
|
|
dir.closeFile(src, newFile);
|
|
|
|
|
|
- // replicate blocks of this file.
|
|
|
|
checkReplicationFactor(newFile);
|
|
checkReplicationFactor(newFile);
|
|
-
|
|
|
|
- NameNode.stateChangeLog.info("DIR* NameSystem.internalReleaseCreate: " +
|
|
|
|
- src + " is no longer written to by " +
|
|
|
|
- holder);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ synchronized void commitBlockSynchronization(Block lastblock,
|
|
|
|
+ long newgenerationstamp, long newlength,
|
|
|
|
+ boolean closeFile, boolean deleteblock, DatanodeID[] newtargets
|
|
|
|
+ ) throws IOException {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("commitBlockSynchronization(lastblock=" + lastblock
|
|
|
|
+ + ", newgenerationstamp=" + newgenerationstamp
|
|
|
|
+ + ", newlength=" + newlength
|
|
|
|
+ + ", newtargets=" + Arrays.asList(newtargets) + ")");
|
|
|
|
+ }
|
|
|
|
+ BlockInfo blockinfo = blocksMap.getStoredBlock(lastblock);
|
|
|
|
+ if (blockinfo == null) {
|
|
|
|
+ throw new IOException("Block (=" + lastblock + ") not found");
|
|
|
|
+ }
|
|
|
|
+ INodeFile iFile = blockinfo.getINode();
|
|
|
|
+ if (!iFile.isUnderConstruction()) {
|
|
|
|
+ throw new IOException("Unexpected block (=" + lastblock
|
|
|
|
+ + ") since the file (=" + iFile.getLocalName()
|
|
|
|
+ + ") is not under construction");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //update block info
|
|
|
|
+ if (newtargets.length > 0) {
|
|
|
|
+ DatanodeDescriptor[] descriptors = new DatanodeDescriptor[newtargets.length];
|
|
|
|
+ for(int i = 0; i < newtargets.length; i++) {
|
|
|
|
+ descriptors[i] = getDatanode(newtargets[i]);
|
|
|
|
+ }
|
|
|
|
+ blockinfo.update(newgenerationstamp, newlength, descriptors);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // If this commit does not want to close the file, just persist
|
|
|
|
+ // block locations and return
|
|
|
|
+ INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)iFile;
|
|
|
|
+ String src = leaseManager.findPath(pendingFile);
|
|
|
|
+ if (deleteblock) {
|
|
|
|
+ dir.removeBlock(src, pendingFile, lastblock);
|
|
|
|
+ }
|
|
|
|
+ if (!closeFile) {
|
|
|
|
+ dir.persistBlocks(src, pendingFile);
|
|
|
|
+ getEditLog().logSync();
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //remove lease, close file
|
|
|
|
+ finalizeINodeFileUnderConstruction(src, pendingFile);
|
|
|
|
+ getEditLog().logSync();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Renew the lease(s) held by the given client
|
|
* Renew the lease(s) held by the given client
|
|
*/
|
|
*/
|
|
@@ -1970,6 +2019,10 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
nodeinfo.updateHeartbeat(capacity, dfsUsed, remaining, xceiverCount);
|
|
nodeinfo.updateHeartbeat(capacity, dfsUsed, remaining, xceiverCount);
|
|
updateStats(nodeinfo, true);
|
|
updateStats(nodeinfo, true);
|
|
|
|
|
|
|
|
+ //check lease recovery
|
|
|
|
+ if (cmd == null) {
|
|
|
|
+ cmd = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
|
|
|
|
+ }
|
|
//check pending replication
|
|
//check pending replication
|
|
if (cmd == null) {
|
|
if (cmd == null) {
|
|
cmd = nodeinfo.getReplicationCommand(
|
|
cmd = nodeinfo.getReplicationCommand(
|
|
@@ -4187,6 +4240,15 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
|
|
return generationStamp.getStamp();
|
|
return generationStamp.getStamp();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Increments, logs and then returns the stamp
|
|
|
|
+ */
|
|
|
|
+ long nextGenerationStamp() {
|
|
|
|
+ long gs = generationStamp.nextStamp();
|
|
|
|
+ getEditLog().logGenerationStamp(gs);
|
|
|
|
+ return gs;
|
|
|
|
+ }
|
|
|
|
+
|
|
// rename was successful. If any part of the renamed subtree had
|
|
// rename was successful. If any part of the renamed subtree had
|
|
// files that were being written to, update with new filename.
|
|
// files that were being written to, update with new filename.
|
|
//
|
|
//
|