HDFS-7056. Snapshot support for truncate. Contributed by Konstantin Shvachko and Plamen Jeliazkov.

This commit is contained in:
Konstantin V Shvachko 2015-01-24 16:06:41 -08:00
parent 6ff9bde714
commit 00a7ebab22
38 changed files with 1351 additions and 391 deletions

View File

@ -21,6 +21,8 @@ Release 2.7.0 - UNRELEASED
HDFS-3107. Introduce truncate. (Plamen Jeliazkov via shv) HDFS-3107. Introduce truncate. (Plamen Jeliazkov via shv)
HDFS-7056. Snapshot support for truncate. (Plamen Jeliazkov and shv)
IMPROVEMENTS IMPROVEMENTS
HDFS-7055. Add tracing to DFSInputStream (cmccabe) HDFS-7055. Add tracing to DFSInputStream (cmccabe)

View File

@ -537,7 +537,7 @@ public interface ClientProtocol {
* @param src existing file * @param src existing file
* @param newLength the target size * @param newLength the target size
* *
* @return true if and client does not need to wait for block recovery, * @return true if client does not need to wait for block recovery,
* false if client needs to wait for block recovery. * false if client needs to wait for block recovery.
* *
* @throws AccessControlException If access is denied * @throws AccessControlException If access is denied

View File

@ -76,12 +76,12 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
final String storageID; final String storageID;
try { try {
storageID = impl.updateReplicaUnderRecovery( storageID = impl.updateReplicaUnderRecovery(
PBHelper.convert(request.getBlock()), PBHelper.convert(request.getBlock()), request.getRecoveryId(),
request.getRecoveryId(), request.getNewLength()); request.getNewBlockId(), request.getNewLength());
} catch (IOException e) { } catch (IOException e) {
throw new ServiceException(e); throw new ServiceException(e);
} }
return UpdateReplicaUnderRecoveryResponseProto.newBuilder() return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
.setStorageUuid(storageID).build(); .setStorageUuid(storageID).build();
} }
} }

View File

@ -102,11 +102,12 @@ public class InterDatanodeProtocolTranslatorPB implements
@Override @Override
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock, public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long newLength) throws IOException { long recoveryId, long newBlockId, long newLength) throws IOException {
UpdateReplicaUnderRecoveryRequestProto req = UpdateReplicaUnderRecoveryRequestProto req =
UpdateReplicaUnderRecoveryRequestProto.newBuilder() UpdateReplicaUnderRecoveryRequestProto.newBuilder()
.setBlock(PBHelper.convert(oldBlock)) .setBlock(PBHelper.convert(oldBlock))
.setNewLength(newLength).setRecoveryId(recoveryId).build(); .setNewLength(newLength).setNewBlockId(newBlockId)
.setRecoveryId(recoveryId).build();
try { try {
return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
).getStorageUuid(); ).getStorageUuid();

View File

@ -608,16 +608,19 @@ public class PBHelper {
return null; return null;
} }
LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b); LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
return RecoveringBlockProto.newBuilder().setBlock(lb) RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
.setNewGenStamp(b.getNewGenerationStamp()) builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
.setTruncateFlag(b.getTruncateFlag()).build(); if(b.getNewBlock() != null)
builder.setTruncateBlock(PBHelper.convert(b.getNewBlock()));
return builder.build();
} }
public static RecoveringBlock convert(RecoveringBlockProto b) { public static RecoveringBlock convert(RecoveringBlockProto b) {
ExtendedBlock block = convert(b.getBlock().getB()); ExtendedBlock block = convert(b.getBlock().getB());
DatanodeInfo[] locs = convert(b.getBlock().getLocsList()); DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
return new RecoveringBlock(block, locs, b.getNewGenStamp(), return (b.hasTruncateBlock()) ?
b.getTruncateFlag()); new RecoveringBlock(block, locs, PBHelper.convert(b.getTruncateBlock())) :
new RecoveringBlock(block, locs, b.getNewGenStamp());
} }
public static DatanodeInfoProto.AdminState convert( public static DatanodeInfoProto.AdminState convert(

View File

@ -54,6 +54,11 @@ public class BlockInfoUnderConstruction extends BlockInfo {
*/ */
private long blockRecoveryId = 0; private long blockRecoveryId = 0;
/**
* The block source to use in the event of copy-on-write truncate.
*/
private Block truncateBlock;
/** /**
* ReplicaUnderConstruction contains information about replicas while * ReplicaUnderConstruction contains information about replicas while
* they are under construction. * they are under construction.
@ -229,6 +234,15 @@ public class BlockInfoUnderConstruction extends BlockInfo {
return blockRecoveryId; return blockRecoveryId;
} }
/** Get recover block */
public Block getTruncateBlock() {
return truncateBlock;
}
public void setTruncateBlock(Block recoveryBlock) {
this.truncateBlock = recoveryBlock;
}
/** /**
* Process the recorded replicas. When about to commit or finish the * Process the recorded replicas. When about to commit or finish the
* pipeline recovery sort out bad replicas. * pipeline recovery sort out bad replicas.
@ -273,11 +287,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* make it primary. * make it primary.
*/ */
public void initializeBlockRecovery(long recoveryId) { public void initializeBlockRecovery(long recoveryId) {
initializeBlockRecovery(BlockUCState.UNDER_RECOVERY, recoveryId); setBlockUCState(BlockUCState.UNDER_RECOVERY);
}
public void initializeBlockRecovery(BlockUCState s, long recoveryId) {
setBlockUCState(s);
blockRecoveryId = recoveryId; blockRecoveryId = recoveryId;
if (replicas.size() == 0) { if (replicas.size() == 0) {
NameNode.blockStateChangeLog.warn("BLOCK*" NameNode.blockStateChangeLog.warn("BLOCK*"

View File

@ -700,13 +700,14 @@ public class BlockManager {
* The client is supposed to allocate a new block with the next call. * The client is supposed to allocate a new block with the next call.
* *
* @param bc file * @param bc file
* @param bytesToRemove num of bytes to remove from block
* @return the last block locations if the block is partial or null otherwise * @return the last block locations if the block is partial or null otherwise
*/ */
public LocatedBlock convertLastBlockToUnderConstruction( public LocatedBlock convertLastBlockToUnderConstruction(
BlockCollection bc) throws IOException { BlockCollection bc, long bytesToRemove) throws IOException {
BlockInfo oldBlock = bc.getLastBlock(); BlockInfo oldBlock = bc.getLastBlock();
if(oldBlock == null || if(oldBlock == null ||
bc.getPreferredBlockSize() == oldBlock.getNumBytes()) bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
return null; return null;
assert oldBlock == getStoredBlock(oldBlock) : assert oldBlock == getStoredBlock(oldBlock) :
"last block of the file is not in blocksMap"; "last block of the file is not in blocksMap";

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.*; import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
import org.apache.hadoop.hdfs.server.namenode.CachedBlock; import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
@ -1432,26 +1431,37 @@ public class DatanodeManager {
recoveryLocations.add(storages[i]); recoveryLocations.add(storages[i]);
} }
} }
// If we are performing a truncate recovery than set recovery fields
// to old block.
boolean truncateRecovery = b.getTruncateBlock() != null;
boolean copyOnTruncateRecovery = truncateRecovery &&
b.getTruncateBlock().getBlockId() != b.getBlockId();
ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
new ExtendedBlock(blockPoolId, b);
// If we only get 1 replica after eliminating stale nodes, then choose all // If we only get 1 replica after eliminating stale nodes, then choose all
// replicas for recovery and let the primary data node handle failures. // replicas for recovery and let the primary data node handle failures.
DatanodeInfo[] recoveryInfos;
if (recoveryLocations.size() > 1) { if (recoveryLocations.size() > 1) {
if (recoveryLocations.size() != storages.length) { if (recoveryLocations.size() != storages.length) {
LOG.info("Skipped stale nodes for recovery : " + LOG.info("Skipped stale nodes for recovery : " +
(storages.length - recoveryLocations.size())); (storages.length - recoveryLocations.size()));
} }
boolean isTruncate = b.getBlockUCState().equals( recoveryInfos =
HdfsServerConstants.BlockUCState.BEING_TRUNCATED); DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
brCommand.add(new RecoveringBlock(
new ExtendedBlock(blockPoolId, b),
DatanodeStorageInfo.toDatanodeInfos(recoveryLocations),
b.getBlockRecoveryId(), isTruncate));
} else { } else {
// If too many replicas are stale, then choose all replicas to participate // If too many replicas are stale, then choose all replicas to participate
// in block recovery. // in block recovery.
brCommand.add(new RecoveringBlock( recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
new ExtendedBlock(blockPoolId, b), }
DatanodeStorageInfo.toDatanodeInfos(storages), if(truncateRecovery) {
b.getBlockRecoveryId())); Block recoveryBlock = (copyOnTruncateRecovery) ? b :
b.getTruncateBlock();
brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
recoveryBlock));
} else {
brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
b.getBlockRecoveryId()));
} }
} }
return new DatanodeCommand[] { brCommand }; return new DatanodeCommand[] { brCommand };

View File

@ -299,13 +299,6 @@ public final class HdfsServerConstants {
* which synchronizes the existing replicas contents. * which synchronizes the existing replicas contents.
*/ */
UNDER_RECOVERY, UNDER_RECOVERY,
/**
* The block is being truncated.<br>
* When a file is truncated its last block may need to be truncated
* and needs to go through a recovery procedure,
* which synchronizes the existing replicas contents.
*/
BEING_TRUNCATED,
/** /**
* The block is committed.<br> * The block is committed.<br>
* The client reported that all bytes are written to data-nodes * The client reported that all bytes are written to data-nodes

View File

@ -2537,14 +2537,16 @@ public class DataNode extends ReconfigurableBase
*/ */
@Override // InterDatanodeProtocol @Override // InterDatanodeProtocol
public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock, public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
final long recoveryId, final long newLength) throws IOException { final long recoveryId, final long newBlockId, final long newLength)
throws IOException {
final String storageID = data.updateReplicaUnderRecovery(oldBlock, final String storageID = data.updateReplicaUnderRecovery(oldBlock,
recoveryId, newLength); recoveryId, newBlockId, newLength);
// Notify the namenode of the updated block info. This is important // Notify the namenode of the updated block info. This is important
// for HA, since otherwise the standby node may lose track of the // for HA, since otherwise the standby node may lose track of the
// block locations until the next block report. // block locations until the next block report.
ExtendedBlock newBlock = new ExtendedBlock(oldBlock); ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
newBlock.setGenerationStamp(recoveryId); newBlock.setGenerationStamp(recoveryId);
newBlock.setBlockId(newBlockId);
newBlock.setNumBytes(newLength); newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "", storageID); notifyNamenodeReceivedBlock(newBlock, "", storageID);
return storageID; return storageID;
@ -2566,10 +2568,12 @@ public class DataNode extends ReconfigurableBase
this.rInfo = rInfo; this.rInfo = rInfo;
} }
void updateReplicaUnderRecovery(String bpid, long recoveryId, long newLength void updateReplicaUnderRecovery(String bpid, long recoveryId,
) throws IOException { long newBlockId, long newLength)
throws IOException {
final ExtendedBlock b = new ExtendedBlock(bpid, rInfo); final ExtendedBlock b = new ExtendedBlock(bpid, rInfo);
storageID = datanode.updateReplicaUnderRecovery(b, recoveryId, newLength); storageID = datanode.updateReplicaUnderRecovery(b, recoveryId, newBlockId,
newLength);
} }
@Override @Override
@ -2651,8 +2655,12 @@ public class DataNode extends ReconfigurableBase
final String bpid = block.getBlockPoolId(); final String bpid = block.getBlockPoolId();
DatanodeProtocolClientSideTranslatorPB nn = DatanodeProtocolClientSideTranslatorPB nn =
getActiveNamenodeForBP(block.getBlockPoolId()); getActiveNamenodeForBP(block.getBlockPoolId());
long recoveryId = rBlock.getNewGenerationStamp(); long recoveryId = rBlock.getNewGenerationStamp();
boolean isTruncateRecovery = rBlock.getNewBlock() != null;
long blockId = (isTruncateRecovery) ?
rBlock.getNewBlock().getBlockId() : block.getBlockId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("block=" + block + ", (length=" + block.getNumBytes() LOG.debug("block=" + block + ", (length=" + block.getNumBytes()
+ "), syncList=" + syncList); + "), syncList=" + syncList);
@ -2686,7 +2694,7 @@ public class DataNode extends ReconfigurableBase
// Calculate list of nodes that will participate in the recovery // Calculate list of nodes that will participate in the recovery
// and the new block size // and the new block size
List<BlockRecord> participatingList = new ArrayList<BlockRecord>(); List<BlockRecord> participatingList = new ArrayList<BlockRecord>();
final ExtendedBlock newBlock = new ExtendedBlock(bpid, block.getBlockId(), final ExtendedBlock newBlock = new ExtendedBlock(bpid, blockId,
-1, recoveryId); -1, recoveryId);
switch(bestState) { switch(bestState) {
case FINALIZED: case FINALIZED:
@ -2698,10 +2706,7 @@ public class DataNode extends ReconfigurableBase
r.rInfo.getNumBytes() == finalizedLength) r.rInfo.getNumBytes() == finalizedLength)
participatingList.add(r); participatingList.add(r);
} }
if(rBlock.getTruncateFlag()) newBlock.setNumBytes(finalizedLength);
newBlock.setNumBytes(rBlock.getBlock().getNumBytes());
else
newBlock.setNumBytes(finalizedLength);
break; break;
case RBW: case RBW:
case RWR: case RWR:
@ -2713,21 +2718,21 @@ public class DataNode extends ReconfigurableBase
participatingList.add(r); participatingList.add(r);
} }
} }
if(rBlock.getTruncateFlag()) newBlock.setNumBytes(minLength);
newBlock.setNumBytes(rBlock.getBlock().getNumBytes());
else
newBlock.setNumBytes(minLength);
break; break;
case RUR: case RUR:
case TEMPORARY: case TEMPORARY:
assert false : "bad replica state: " + bestState; assert false : "bad replica state: " + bestState;
} }
if(isTruncateRecovery)
newBlock.setNumBytes(rBlock.getNewBlock().getNumBytes());
List<DatanodeID> failedList = new ArrayList<DatanodeID>(); List<DatanodeID> failedList = new ArrayList<DatanodeID>();
final List<BlockRecord> successList = new ArrayList<BlockRecord>(); final List<BlockRecord> successList = new ArrayList<BlockRecord>();
for(BlockRecord r : participatingList) { for(BlockRecord r : participatingList) {
try { try {
r.updateReplicaUnderRecovery(bpid, recoveryId, newBlock.getNumBytes()); r.updateReplicaUnderRecovery(bpid, recoveryId, blockId,
newBlock.getNumBytes());
successList.add(r); successList.add(r);
} catch (IOException e) { } catch (IOException e) {
InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock=" InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock="

View File

@ -419,7 +419,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
* @return the ID of storage that stores the block * @return the ID of storage that stores the block
*/ */
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock, public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long newLength) throws IOException; long recoveryId, long newBlockId, long newLength) throws IOException;
/** /**
* add new block pool ID * add new block pool ID
@ -516,4 +516,4 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
*/ */
public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block, public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
StorageType targetStorageType) throws IOException; StorageType targetStorageType) throws IOException;
} }

View File

@ -687,6 +687,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId); final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
final File dstFile = new File(destDir, srcFile.getName()); final File dstFile = new File(destDir, srcFile.getName());
final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp); final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum);
}
static File[] copyBlockFiles(File srcMeta, File srcFile, File dstMeta,
File dstFile, boolean calculateChecksum)
throws IOException {
if (calculateChecksum) { if (calculateChecksum) {
computeChecksum(srcMeta, dstMeta, srcFile); computeChecksum(srcMeta, dstMeta, srcFile);
} else { } else {
@ -2217,6 +2223,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
public synchronized String updateReplicaUnderRecovery( public synchronized String updateReplicaUnderRecovery(
final ExtendedBlock oldBlock, final ExtendedBlock oldBlock,
final long recoveryId, final long recoveryId,
final long newBlockId,
final long newlength) throws IOException { final long newlength) throws IOException {
//get replica //get replica
final String bpid = oldBlock.getBlockPoolId(); final String bpid = oldBlock.getBlockPoolId();
@ -2249,13 +2256,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
//update replica //update replica
final FinalizedReplica finalized = updateReplicaUnderRecovery(oldBlock final FinalizedReplica finalized = updateReplicaUnderRecovery(oldBlock
.getBlockPoolId(), (ReplicaUnderRecovery) replica, recoveryId, newlength); .getBlockPoolId(), (ReplicaUnderRecovery) replica, recoveryId,
assert finalized.getBlockId() == oldBlock.getBlockId() newBlockId, newlength);
&& finalized.getGenerationStamp() == recoveryId
&& finalized.getNumBytes() == newlength boolean copyTruncate = newBlockId != oldBlock.getBlockId();
: "Replica information mismatched: oldBlock=" + oldBlock if(!copyTruncate) {
+ ", recoveryId=" + recoveryId + ", newlength=" + newlength assert finalized.getBlockId() == oldBlock.getBlockId()
+ ", finalized=" + finalized; && finalized.getGenerationStamp() == recoveryId
&& finalized.getNumBytes() == newlength
: "Replica information mismatched: oldBlock=" + oldBlock
+ ", recoveryId=" + recoveryId + ", newlength=" + newlength
+ ", newBlockId=" + newBlockId + ", finalized=" + finalized;
} else {
assert finalized.getBlockId() == oldBlock.getBlockId()
&& finalized.getGenerationStamp() == oldBlock.getGenerationStamp()
&& finalized.getNumBytes() == oldBlock.getNumBytes()
: "Finalized and old information mismatched: oldBlock=" + oldBlock
+ ", genStamp=" + oldBlock.getGenerationStamp()
+ ", len=" + oldBlock.getNumBytes()
+ ", finalized=" + finalized;
}
//check replica files after update //check replica files after update
checkReplicaFiles(finalized); checkReplicaFiles(finalized);
@ -2268,6 +2288,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
String bpid, String bpid,
ReplicaUnderRecovery rur, ReplicaUnderRecovery rur,
long recoveryId, long recoveryId,
long newBlockId,
long newlength) throws IOException { long newlength) throws IOException {
//check recovery id //check recovery id
if (rur.getRecoveryID() != recoveryId) { if (rur.getRecoveryID() != recoveryId) {
@ -2275,26 +2296,63 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
+ ", rur=" + rur); + ", rur=" + rur);
} }
boolean copyOnTruncate = newBlockId > 0L && rur.getBlockId() != newBlockId;
File blockFile;
File metaFile;
// bump rur's GS to be recovery id // bump rur's GS to be recovery id
bumpReplicaGS(rur, recoveryId); if(!copyOnTruncate) {
bumpReplicaGS(rur, recoveryId);
blockFile = rur.getBlockFile();
metaFile = rur.getMetaFile();
} else {
File[] copiedReplicaFiles =
copyReplicaWithNewBlockIdAndGS(rur, bpid, newBlockId, recoveryId);
blockFile = copiedReplicaFiles[1];
metaFile = copiedReplicaFiles[0];
}
//update length //update length
final File replicafile = rur.getBlockFile();
if (rur.getNumBytes() < newlength) { if (rur.getNumBytes() < newlength) {
throw new IOException("rur.getNumBytes() < newlength = " + newlength throw new IOException("rur.getNumBytes() < newlength = " + newlength
+ ", rur=" + rur); + ", rur=" + rur);
} }
if (rur.getNumBytes() > newlength) { if (rur.getNumBytes() > newlength) {
rur.unlinkBlock(1); rur.unlinkBlock(1);
truncateBlock(replicafile, rur.getMetaFile(), rur.getNumBytes(), newlength); truncateBlock(blockFile, metaFile, rur.getNumBytes(), newlength);
// update RUR with the new length if(!copyOnTruncate) {
rur.setNumBytes(newlength); // update RUR with the new length
rur.setNumBytes(newlength);
} else {
// Copying block to a new block with new blockId.
// Not truncating original block.
ReplicaBeingWritten newReplicaInfo = new ReplicaBeingWritten(
newBlockId, recoveryId, rur.getVolume(), blockFile.getParentFile(),
newlength);
newReplicaInfo.setNumBytes(newlength);
volumeMap.add(bpid, newReplicaInfo);
finalizeReplica(bpid, newReplicaInfo);
}
} }
// finalize the block // finalize the block
return finalizeReplica(bpid, rur); return finalizeReplica(bpid, rur);
} }
private File[] copyReplicaWithNewBlockIdAndGS(
ReplicaUnderRecovery replicaInfo, String bpid, long newBlkId, long newGS)
throws IOException {
String blockFileName = Block.BLOCK_FILE_PREFIX + newBlkId;
FsVolumeReference v = volumes.getNextVolume(
replicaInfo.getVolume().getStorageType(), replicaInfo.getNumBytes());
final File tmpDir = ((FsVolumeImpl) v.getVolume())
.getBlockPoolSlice(bpid).getTmpDir();
final File destDir = DatanodeUtil.idToBlockDir(tmpDir, newBlkId);
final File dstBlockFile = new File(destDir, blockFileName);
final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
return copyBlockFiles(replicaInfo.getMetaFile(), replicaInfo.getBlockFile(),
dstMetaFile, dstBlockFile, true);
}
@Override // FsDatasetSpi @Override // FsDatasetSpi
public synchronized long getReplicaVisibleLength(final ExtendedBlock block) public synchronized long getReplicaVisibleLength(final ExtendedBlock block)
throws IOException { throws IOException {

View File

@ -424,7 +424,7 @@ class FSDirStatAndListingOp {
fileNode.computeFileSizeNotIncludingLastUcBlock() : size; fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks( loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks(
fileNode.getBlocks(), fileSize, isUc, 0L, size, false, fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
inSnapshot, feInfo); inSnapshot, feInfo);
if (loc == null) { if (loc == null) {
loc = new LocatedBlocks(); loc = new LocatedBlocks();

View File

@ -957,18 +957,31 @@ public class FSDirectory implements Closeable {
* Unlike FSNamesystem.truncate, this will not schedule block recovery. * Unlike FSNamesystem.truncate, this will not schedule block recovery.
*/ */
void unprotectedTruncate(String src, String clientName, String clientMachine, void unprotectedTruncate(String src, String clientName, String clientMachine,
long newLength, long mtime) long newLength, long mtime, Block truncateBlock)
throws UnresolvedLinkException, QuotaExceededException, throws UnresolvedLinkException, QuotaExceededException,
SnapshotAccessControlException, IOException { SnapshotAccessControlException, IOException {
INodesInPath iip = getINodesInPath(src, true); INodesInPath iip = getINodesInPath(src, true);
INodeFile file = iip.getLastINode().asFile();
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo(); BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
boolean onBlockBoundary = boolean onBlockBoundary =
unprotectedTruncate(iip, newLength, collectedBlocks, mtime); unprotectedTruncate(iip, newLength, collectedBlocks, mtime);
if(! onBlockBoundary) { if(! onBlockBoundary) {
getFSNamesystem().prepareFileForWrite(src, BlockInfo oldBlock = file.getLastBlock();
iip, clientName, clientMachine, false, false); Block tBlk =
getFSNamesystem().prepareFileForTruncate(iip,
clientName, clientMachine, file.computeFileSize() - newLength,
truncateBlock);
assert Block.matchingIdAndGenStamp(tBlk, truncateBlock) &&
tBlk.getNumBytes() == truncateBlock.getNumBytes() :
"Should be the same block.";
if(oldBlock.getBlockId() != tBlk.getBlockId() &&
!file.isBlockInLatestSnapshot(oldBlock)) {
getBlockManager().removeBlockFromMap(oldBlock);
}
} }
assert onBlockBoundary == (truncateBlock == null) :
"truncateBlock is null iff on block boundary: " + truncateBlock;
getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks); getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
} }
@ -987,7 +1000,8 @@ public class FSDirectory implements Closeable {
/** /**
* Truncate has the following properties: * Truncate has the following properties:
* 1.) Any block deletions occur now. * 1.) Any block deletions occur now.
* 2.) INode length is truncated now clients can only read up to new length. * 2.) INode length is truncated now new clients can only read up to
* the truncated length.
* 3.) INode will be set to UC and lastBlock set to UNDER_RECOVERY. * 3.) INode will be set to UC and lastBlock set to UNDER_RECOVERY.
* 4.) NN will trigger DN truncation recovery and waits for DNs to report. * 4.) NN will trigger DN truncation recovery and waits for DNs to report.
* 5.) File is considered UNDER_RECOVERY until truncation recovery completes. * 5.) File is considered UNDER_RECOVERY until truncation recovery completes.
@ -1000,20 +1014,16 @@ public class FSDirectory implements Closeable {
long mtime) throws IOException { long mtime) throws IOException {
assert hasWriteLock(); assert hasWriteLock();
INodeFile file = iip.getLastINode().asFile(); INodeFile file = iip.getLastINode().asFile();
int latestSnapshot = iip.getLatestSnapshotId();
file.recordModification(latestSnapshot, true);
long oldDiskspace = file.diskspaceConsumed(); long oldDiskspace = file.diskspaceConsumed();
long remainingLength = long remainingLength =
file.collectBlocksBeyondMax(newLength, collectedBlocks); file.collectBlocksBeyondMax(newLength, collectedBlocks);
file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
file.setModificationTime(mtime); file.setModificationTime(mtime);
updateCount(iip, 0, file.diskspaceConsumed() - oldDiskspace, true); updateCount(iip, 0, file.diskspaceConsumed() - oldDiskspace, true);
// If on block boundary, then return // return whether on a block boundary
long lastBlockDelta = remainingLength - newLength; return (remainingLength - newLength) == 0;
if(lastBlockDelta == 0)
return true;
// Set new last block length
BlockInfo lastBlock = file.getLastBlock();
assert lastBlock.getNumBytes() - lastBlockDelta > 0 : "wrong block size";
lastBlock.setNumBytes(lastBlock.getNumBytes() - lastBlockDelta);
return false;
} }
/** /**

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@ -902,13 +903,14 @@ public class FSEditLog implements LogsPurgeable {
* Add truncate file record to edit log * Add truncate file record to edit log
*/ */
void logTruncate(String src, String clientName, String clientMachine, void logTruncate(String src, String clientName, String clientMachine,
long size, long timestamp) { long size, long timestamp, Block truncateBlock) {
TruncateOp op = TruncateOp.getInstance(cache.get()) TruncateOp op = TruncateOp.getInstance(cache.get())
.setPath(src) .setPath(src)
.setClientName(clientName) .setClientName(clientName)
.setClientMachine(clientMachine) .setClientMachine(clientMachine)
.setNewLength(size) .setNewLength(size)
.setTimestamp(timestamp); .setTimestamp(timestamp)
.setTruncateBlock(truncateBlock);
logEdit(op); logEdit(op);
} }

View File

@ -861,7 +861,8 @@ public class FSEditLogLoader {
case OP_TRUNCATE: { case OP_TRUNCATE: {
TruncateOp truncateOp = (TruncateOp) op; TruncateOp truncateOp = (TruncateOp) op;
fsDir.unprotectedTruncate(truncateOp.src, truncateOp.clientName, fsDir.unprotectedTruncate(truncateOp.src, truncateOp.clientName,
truncateOp.clientMachine, truncateOp.newLength, truncateOp.timestamp); truncateOp.clientMachine, truncateOp.newLength, truncateOp.timestamp,
truncateOp.truncateBlock);
break; break;
} }
case OP_SET_STORAGE_POLICY: { case OP_SET_STORAGE_POLICY: {

View File

@ -2613,6 +2613,7 @@ public abstract class FSEditLogOp {
String clientMachine; String clientMachine;
long newLength; long newLength;
long timestamp; long timestamp;
Block truncateBlock;
private TruncateOp() { private TruncateOp() {
super(OP_TRUNCATE); super(OP_TRUNCATE);
@ -2656,6 +2657,11 @@ public abstract class FSEditLogOp {
return this; return this;
} }
TruncateOp setTruncateBlock(Block truncateBlock) {
this.truncateBlock = truncateBlock;
return this;
}
@Override @Override
void readFields(DataInputStream in, int logVersion) throws IOException { void readFields(DataInputStream in, int logVersion) throws IOException {
src = FSImageSerialization.readString(in); src = FSImageSerialization.readString(in);
@ -2663,6 +2669,10 @@ public abstract class FSEditLogOp {
clientMachine = FSImageSerialization.readString(in); clientMachine = FSImageSerialization.readString(in);
newLength = FSImageSerialization.readLong(in); newLength = FSImageSerialization.readLong(in);
timestamp = FSImageSerialization.readLong(in); timestamp = FSImageSerialization.readLong(in);
Block[] blocks =
FSImageSerialization.readCompactBlockArray(in, logVersion);
assert blocks.length <= 1 : "Truncate op should have 1 or 0 blocks";
truncateBlock = (blocks.length == 0) ? null : blocks[0];
} }
@Override @Override
@ -2672,6 +2682,12 @@ public abstract class FSEditLogOp {
FSImageSerialization.writeString(clientMachine, out); FSImageSerialization.writeString(clientMachine, out);
FSImageSerialization.writeLong(newLength, out); FSImageSerialization.writeLong(newLength, out);
FSImageSerialization.writeLong(timestamp, out); FSImageSerialization.writeLong(timestamp, out);
int size = truncateBlock != null ? 1 : 0;
Block[] blocks = new Block[size];
if (truncateBlock != null) {
blocks[0] = truncateBlock;
}
FSImageSerialization.writeCompactBlockArray(blocks, out);
} }
@Override @Override
@ -2683,6 +2699,8 @@ public abstract class FSEditLogOp {
Long.toString(newLength)); Long.toString(newLength));
XMLUtils.addSaxString(contentHandler, "TIMESTAMP", XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
Long.toString(timestamp)); Long.toString(timestamp));
if(truncateBlock != null)
FSEditLogOp.blockToXml(contentHandler, truncateBlock);
} }
@Override @Override
@ -2692,6 +2710,8 @@ public abstract class FSEditLogOp {
this.clientMachine = st.getValue("CLIENTMACHINE"); this.clientMachine = st.getValue("CLIENTMACHINE");
this.newLength = Long.parseLong(st.getValue("NEWLENGTH")); this.newLength = Long.parseLong(st.getValue("NEWLENGTH"));
this.timestamp = Long.parseLong(st.getValue("TIMESTAMP")); this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
if (st.hasChildren("BLOCK"))
this.truncateBlock = FSEditLogOp.blockFromXml(st);
} }
@Override @Override
@ -2707,6 +2727,8 @@ public abstract class FSEditLogOp {
builder.append(newLength); builder.append(newLength);
builder.append(", timestamp="); builder.append(", timestamp=");
builder.append(timestamp); builder.append(timestamp);
builder.append(", truncateBlock=");
builder.append(truncateBlock);
builder.append(", opCode="); builder.append(", opCode=");
builder.append(opCode); builder.append(opCode);
builder.append(", txid="); builder.append(", txid=");

View File

@ -1800,8 +1800,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
: dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip); : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
final LocatedBlocks blocks = blockManager.createLocatedBlocks( final LocatedBlocks blocks = blockManager.createLocatedBlocks(
inode.getBlocks(), fileSize, isUc, offset, length, needBlockToken, inode.getBlocks(iip.getPathSnapshotId()), fileSize,
iip.isSnapshot(), feInfo); isUc, offset, length, needBlockToken, iip.isSnapshot(), feInfo);
// Set caching information for the located blocks. // Set caching information for the located blocks.
for (LocatedBlock lb : blocks.getLocatedBlocks()) { for (LocatedBlock lb : blocks.getLocatedBlocks()) {
@ -1939,7 +1939,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* Truncation at block boundary is atomic, otherwise it requires * Truncation at block boundary is atomic, otherwise it requires
* block recovery to truncate the last block of the file. * block recovery to truncate the last block of the file.
* *
* @return true if and client does not need to wait for block recovery, * @return true if client does not need to wait for block recovery,
* false if client needs to wait for block recovery. * false if client needs to wait for block recovery.
*/ */
boolean truncate(String src, long newLength, boolean truncate(String src, long newLength,
@ -2001,44 +2001,119 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
dir.checkPathAccess(pc, iip, FsAction.WRITE); dir.checkPathAccess(pc, iip, FsAction.WRITE);
} }
INodeFile file = iip.getLastINode().asFile(); INodeFile file = iip.getLastINode().asFile();
// Data will be lost after truncate occurs so it cannot support snapshots.
if(file.isInLatestSnapshot(iip.getLatestSnapshotId()))
throw new HadoopIllegalArgumentException(
"Cannot truncate file with snapshot.");
// Opening an existing file for write. May need lease recovery. // Opening an existing file for write. May need lease recovery.
recoverLeaseInternal(iip, src, clientName, clientMachine, false); recoverLeaseInternal(iip, src, clientName, clientMachine, false);
// Refresh INode as the file could have been closed
iip = dir.getINodesInPath4Write(src, true);
file = INodeFile.valueOf(iip.getLastINode(), src); file = INodeFile.valueOf(iip.getLastINode(), src);
// Truncate length check. // Truncate length check.
long oldLength = file.computeFileSize(); long oldLength = file.computeFileSize();
if(oldLength == newLength) if(oldLength == newLength) {
return true; return true;
if(oldLength < newLength) }
if(oldLength < newLength) {
throw new HadoopIllegalArgumentException( throw new HadoopIllegalArgumentException(
"Cannot truncate to a larger file size. Current size: " + oldLength + "Cannot truncate to a larger file size. Current size: " + oldLength +
", truncate size: " + newLength + "."); ", truncate size: " + newLength + ".");
}
// Perform INodeFile truncation. // Perform INodeFile truncation.
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo(); BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
boolean onBlockBoundary = dir.truncate(iip, newLength, boolean onBlockBoundary = dir.truncate(iip, newLength,
collectedBlocks, mtime); collectedBlocks, mtime);
Block truncateBlock = null;
if(! onBlockBoundary) { if(! onBlockBoundary) {
// Open file for write, but don't log into edits // Open file for write, but don't log into edits
prepareFileForWrite(src, iip, clientName, clientMachine, false, false); long lastBlockDelta = file.computeFileSize() - newLength;
file = INodeFile.valueOf(dir.getINode4Write(src), src); assert lastBlockDelta > 0 : "delta is 0 only if on block bounday";
initializeBlockRecovery(file); truncateBlock = prepareFileForTruncate(iip, clientName, clientMachine,
lastBlockDelta, null);
} }
getEditLog().logTruncate(src, clientName, clientMachine, newLength, mtime); getEditLog().logTruncate(src, clientName, clientMachine, newLength, mtime,
truncateBlock);
removeBlocks(collectedBlocks); removeBlocks(collectedBlocks);
return onBlockBoundary; return onBlockBoundary;
} }
void initializeBlockRecovery(INodeFile inodeFile) throws IOException { /**
BlockInfo lastBlock = inodeFile.getLastBlock(); * Convert current INode to UnderConstruction.
long recoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(lastBlock)); * Recreate lease.
((BlockInfoUnderConstruction)lastBlock).initializeBlockRecovery( * Create new block for the truncated copy.
BlockUCState.BEING_TRUNCATED, recoveryId); * Schedule truncation of the replicas.
*
* @return the returned block will be written to editLog and passed back into
* this method upon loading.
*/
Block prepareFileForTruncate(INodesInPath iip,
String leaseHolder,
String clientMachine,
long lastBlockDelta,
Block newBlock)
throws IOException {
INodeFile file = iip.getLastINode().asFile();
String src = iip.getPath();
file.recordModification(iip.getLatestSnapshotId());
file.toUnderConstruction(leaseHolder, clientMachine);
assert file.isUnderConstruction() : "inode should be under construction.";
leaseManager.addLease(
file.getFileUnderConstructionFeature().getClientName(), src);
boolean shouldRecoverNow = (newBlock == null);
BlockInfo oldBlock = file.getLastBlock();
boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
if(newBlock == null) {
newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
}
BlockInfoUnderConstruction truncatedBlockUC;
if(shouldCopyOnTruncate) {
// Add new truncateBlock into blocksMap and
// use oldBlock as a source for copy-on-truncate recovery
truncatedBlockUC = new BlockInfoUnderConstruction(newBlock,
file.getBlockReplication());
truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
truncatedBlockUC.setTruncateBlock(oldBlock);
file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
getBlockManager().addBlockCollection(truncatedBlockUC, file);
NameNode.stateChangeLog.info("BLOCK* prepareFileForTruncate: "
+ "Scheduling copy-on-truncate to new size "
+ truncatedBlockUC.getNumBytes() + " new block " + newBlock
+ " old block " + truncatedBlockUC.getTruncateBlock());
} else {
// Use new generation stamp for in-place truncate recovery
blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
oldBlock = file.getLastBlock();
assert !oldBlock.isComplete() : "oldBlock should be under construction";
truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
truncatedBlockUC.getTruncateBlock().setNumBytes(
oldBlock.getNumBytes() - lastBlockDelta);
truncatedBlockUC.getTruncateBlock().setGenerationStamp(
newBlock.getGenerationStamp());
NameNode.stateChangeLog.debug("BLOCK* prepareFileForTruncate: "
+ "Scheduling in-place block truncate to new size "
+ truncatedBlockUC.getTruncateBlock().getNumBytes()
+ " block=" + truncatedBlockUC);
}
if(shouldRecoverNow)
truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
// update the quota: use the preferred block size for UC block
final long diff =
file.getPreferredBlockSize() - truncatedBlockUC.getNumBytes();
dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
return newBlock;
}
/**
* Defines if a replica needs to be copied on truncate or
* can be truncated in place.
*/
boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) {
if(!isUpgradeFinalized()) {
return true;
}
return file.isBlockInLatestSnapshot(blk);
} }
/** /**
@ -2565,7 +2640,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
leaseManager.addLease( leaseManager.addLease(
file.getFileUnderConstructionFeature().getClientName(), src); file.getFileUnderConstructionFeature().getClientName(), src);
LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(file); LocatedBlock ret =
blockManager.convertLastBlockToUnderConstruction(file, 0);
if (ret != null) { if (ret != null) {
// update the quota: use the preferred block size for UC block // update the quota: use the preferred block size for UC block
final long diff = file.getPreferredBlockSize() - ret.getBlockSize(); final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
@ -2628,7 +2704,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return false; return false;
} }
private void recoverLeaseInternal(INodesInPath iip, void recoverLeaseInternal(INodesInPath iip,
String src, String holder, String clientMachine, boolean force) String src, String holder, String clientMachine, boolean force)
throws IOException { throws IOException {
assert hasWriteLock(); assert hasWriteLock();
@ -2690,8 +2766,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
} else { } else {
final BlockInfo lastBlock = file.getLastBlock(); final BlockInfo lastBlock = file.getLastBlock();
if (lastBlock != null if (lastBlock != null
&& (lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY || && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
lastBlock.getBlockUCState() == BlockUCState.BEING_TRUNCATED)) {
throw new RecoveryInProgressException("Recovery in progress, file [" throw new RecoveryInProgressException("Recovery in progress, file ["
+ src + "], " + "lease owner [" + lease.getHolder() + "]"); + src + "], " + "lease owner [" + lease.getHolder() + "]");
} else { } else {
@ -3845,8 +3920,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
throw new AlreadyBeingCreatedException(message); throw new AlreadyBeingCreatedException(message);
case UNDER_CONSTRUCTION: case UNDER_CONSTRUCTION:
case UNDER_RECOVERY: case UNDER_RECOVERY:
case BEING_TRUNCATED:
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock; final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
// determine if last block was intended to be truncated
Block recoveryBlock = uc.getTruncateBlock();
boolean truncateRecovery = recoveryBlock != null;
boolean copyOnTruncate = truncateRecovery &&
recoveryBlock.getBlockId() != uc.getBlockId();
assert !copyOnTruncate ||
recoveryBlock.getBlockId() < uc.getBlockId() &&
recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() &&
recoveryBlock.getNumBytes() > uc.getNumBytes() :
"wrong recoveryBlock";
// setup the last block locations from the blockManager if not known // setup the last block locations from the blockManager if not known
if (uc.getNumExpectedLocations() == 0) { if (uc.getNumExpectedLocations() == 0) {
uc.setExpectedLocations(blockManager.getStorages(lastBlock)); uc.setExpectedLocations(blockManager.getStorages(lastBlock));
@ -3867,9 +3952,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// start recovery of the last block for this file // start recovery of the last block for this file
long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc)); long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile); lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
if (uc.getBlockUCState() != BlockUCState.BEING_TRUNCATED) { if(copyOnTruncate) {
uc.initializeBlockRecovery(blockRecoveryId); uc.setGenerationStamp(blockRecoveryId);
} else if(truncateRecovery) {
recoveryBlock.setGenerationStamp(blockRecoveryId);
} }
uc.initializeBlockRecovery(blockRecoveryId);
leaseManager.renewLease(lease); leaseManager.renewLease(lease);
// Cannot close file right now, since the last block requires recovery. // Cannot close file right now, since the last block requires recovery.
// This may potentially cause infinite loop in lease recovery // This may potentially cause infinite loop in lease recovery
@ -3979,11 +4067,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return true; return true;
} }
void commitBlockSynchronization(ExtendedBlock lastblock, void commitBlockSynchronization(ExtendedBlock oldBlock,
long newgenerationstamp, long newlength, long newgenerationstamp, long newlength,
boolean closeFile, boolean deleteblock, DatanodeID[] newtargets, boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
String[] newtargetstorages) throws IOException { String[] newtargetstorages) throws IOException {
LOG.info("commitBlockSynchronization(lastblock=" + lastblock LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock
+ ", newgenerationstamp=" + newgenerationstamp + ", newgenerationstamp=" + newgenerationstamp
+ ", newlength=" + newlength + ", newlength=" + newlength
+ ", newtargets=" + Arrays.asList(newtargets) + ", newtargets=" + Arrays.asList(newtargets)
@ -4002,17 +4090,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
checkNameNodeSafeMode( checkNameNodeSafeMode(
"Cannot commitBlockSynchronization while in safe mode"); "Cannot commitBlockSynchronization while in safe mode");
final BlockInfo storedBlock = getStoredBlock( final BlockInfo storedBlock = getStoredBlock(
ExtendedBlock.getLocalBlock(lastblock)); ExtendedBlock.getLocalBlock(oldBlock));
if (storedBlock == null) { if (storedBlock == null) {
if (deleteblock) { if (deleteblock) {
// This may be a retry attempt so ignore the failure // This may be a retry attempt so ignore the failure
// to locate the block. // to locate the block.
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Block (=" + lastblock + ") not found"); LOG.debug("Block (=" + oldBlock + ") not found");
} }
return; return;
} else { } else {
throw new IOException("Block (=" + lastblock + ") not found"); throw new IOException("Block (=" + oldBlock + ") not found");
} }
} }
// //
@ -4039,34 +4127,40 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
+ iFile.getFullPathName() + ", likely due to delayed block" + iFile.getFullPathName() + ", likely due to delayed block"
+ " removal"); + " removal");
} }
if (!iFile.isUnderConstruction() || storedBlock.isComplete()) { if ((!iFile.isUnderConstruction() || storedBlock.isComplete()) &&
iFile.getLastBlock().isComplete()) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Unexpected block (=" + lastblock LOG.debug("Unexpected block (=" + oldBlock
+ ") since the file (=" + iFile.getLocalName() + ") since the file (=" + iFile.getLocalName()
+ ") is not under construction"); + ") is not under construction");
} }
return; return;
} }
long recoveryId = BlockInfoUnderConstruction truncatedBlock =
((BlockInfoUnderConstruction)storedBlock).getBlockRecoveryId(); (BlockInfoUnderConstruction) iFile.getLastBlock();
long recoveryId = truncatedBlock.getBlockRecoveryId();
boolean copyTruncate =
truncatedBlock.getBlockId() != storedBlock.getBlockId();
if(recoveryId != newgenerationstamp) { if(recoveryId != newgenerationstamp) {
throw new IOException("The recovery id " + newgenerationstamp throw new IOException("The recovery id " + newgenerationstamp
+ " does not match current recovery id " + " does not match current recovery id "
+ recoveryId + " for block " + lastblock); + recoveryId + " for block " + oldBlock);
} }
if (deleteblock) { if (deleteblock) {
Block blockToDel = ExtendedBlock.getLocalBlock(lastblock); Block blockToDel = ExtendedBlock.getLocalBlock(oldBlock);
boolean remove = iFile.removeLastBlock(blockToDel); boolean remove = iFile.removeLastBlock(blockToDel);
if (remove) { if (remove) {
blockManager.removeBlockFromMap(storedBlock); blockManager.removeBlock(storedBlock);
} }
} }
else { else {
// update last block // update last block
storedBlock.setGenerationStamp(newgenerationstamp); if(!copyTruncate) {
storedBlock.setNumBytes(newlength); storedBlock.setGenerationStamp(newgenerationstamp);
storedBlock.setNumBytes(newlength);
}
// find the DatanodeDescriptor objects // find the DatanodeDescriptor objects
// There should be no locations in the blockManager till now because the // There should be no locations in the blockManager till now because the
@ -4096,7 +4190,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
DatanodeStorageInfo storageInfo = DatanodeStorageInfo storageInfo =
trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i)); trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
if (storageInfo != null) { if (storageInfo != null) {
storageInfo.addBlock(storedBlock); if(copyTruncate) {
storageInfo.addBlock(truncatedBlock);
} else {
storageInfo.addBlock(storedBlock);
}
} }
} }
} }
@ -4106,11 +4204,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
blockManager.getDatanodeManager().getDatanodeStorageInfos( blockManager.getDatanodeManager().getDatanodeStorageInfos(
trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]), trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
trimmedStorages.toArray(new String[trimmedStorages.size()])); trimmedStorages.toArray(new String[trimmedStorages.size()]));
iFile.setLastBlock(storedBlock, trimmedStorageInfos); if(copyTruncate) {
iFile.setLastBlock(truncatedBlock, trimmedStorageInfos);
} else {
iFile.setLastBlock(storedBlock, trimmedStorageInfos);
}
} }
if (closeFile) { if (closeFile) {
src = closeFileCommitBlocks(iFile, storedBlock); if(copyTruncate) {
src = closeFileCommitBlocks(iFile, truncatedBlock);
if(!iFile.isBlockInLatestSnapshot(storedBlock)) {
blockManager.removeBlock(storedBlock);
}
} else {
src = closeFileCommitBlocks(iFile, storedBlock);
}
} else { } else {
// If this commit does not want to close the file, persist blocks // If this commit does not want to close the file, persist blocks
src = iFile.getFullPathName(); src = iFile.getFullPathName();
@ -4121,13 +4230,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
} }
getEditLog().logSync(); getEditLog().logSync();
if (closeFile) { if (closeFile) {
LOG.info("commitBlockSynchronization(newblock=" + lastblock LOG.info("commitBlockSynchronization(oldBlock=" + oldBlock
+ ", file=" + src + ", file=" + src
+ ", newgenerationstamp=" + newgenerationstamp + ", newgenerationstamp=" + newgenerationstamp
+ ", newlength=" + newlength + ", newlength=" + newlength
+ ", newtargets=" + Arrays.asList(newtargets) + ") successful"); + ", newtargets=" + Arrays.asList(newtargets) + ") successful");
} else { } else {
LOG.info("commitBlockSynchronization(" + lastblock + ") successful"); LOG.info("commitBlockSynchronization(" + oldBlock + ") successful");
} }
} }

View File

@ -228,7 +228,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
/** Is this inode in the latest snapshot? */ /** Is this inode in the latest snapshot? */
public final boolean isInLatestSnapshot(final int latestSnapshotId) { public final boolean isInLatestSnapshot(final int latestSnapshotId) {
if (latestSnapshotId == Snapshot.CURRENT_STATE_ID) { if (latestSnapshotId == Snapshot.CURRENT_STATE_ID ||
latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
return false; return false;
} }
// if parent is a reference node, parent must be a renamed node. We can // if parent is a reference node, parent must be a renamed node. We can
@ -817,11 +818,15 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
* @param toDelete the to-be-deleted block * @param toDelete the to-be-deleted block
*/ */
public void addDeleteBlock(Block toDelete) { public void addDeleteBlock(Block toDelete) {
if (toDelete != null) { assert toDelete != null : "toDelete is null";
toDeleteList.add(toDelete); toDeleteList.add(toDelete);
}
} }
public void removeDeleteBlock(Block block) {
assert block != null : "block is null";
toDeleteList.remove(block);
}
/** /**
* Clear {@link BlocksMapUpdateInfo#toDeleteList} * Clear {@link BlocksMapUpdateInfo#toDeleteList}
*/ */

View File

@ -24,7 +24,9 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
@ -304,6 +306,11 @@ public class INodeFile extends INodeWithAdditionalFields
@Override @Override
public void recordModification(final int latestSnapshotId) public void recordModification(final int latestSnapshotId)
throws QuotaExceededException { throws QuotaExceededException {
recordModification(latestSnapshotId, false);
}
public void recordModification(final int latestSnapshotId, boolean withBlocks)
throws QuotaExceededException {
if (isInLatestSnapshot(latestSnapshotId) if (isInLatestSnapshot(latestSnapshotId)
&& !shouldRecordInSrcSnapshot(latestSnapshotId)) { && !shouldRecordInSrcSnapshot(latestSnapshotId)) {
// the file is in snapshot, create a snapshot feature if it does not have // the file is in snapshot, create a snapshot feature if it does not have
@ -312,10 +319,10 @@ public class INodeFile extends INodeWithAdditionalFields
sf = addSnapshotFeature(null); sf = addSnapshotFeature(null);
} }
// record self in the diff list if necessary // record self in the diff list if necessary
sf.getDiffs().saveSelf2Snapshot(latestSnapshotId, this, null); sf.getDiffs().saveSelf2Snapshot(latestSnapshotId, this, null, withBlocks);
} }
} }
public FileDiffList getDiffs() { public FileDiffList getDiffs() {
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature(); FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
if (sf != null) { if (sf != null) {
@ -415,6 +422,20 @@ public class INodeFile extends INodeWithAdditionalFields
return this.blocks; return this.blocks;
} }
/** @return blocks of the file corresponding to the snapshot. */
public BlockInfo[] getBlocks(int snapshot) {
if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
return getBlocks();
FileDiff diff = getDiffs().getDiffById(snapshot);
BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
if(snapshotBlocks != null)
return snapshotBlocks;
// Blocks are not in the current snapshot
// Find next snapshot with blocks present or return current file blocks
snapshotBlocks = getDiffs().findLaterSnapshotBlocks(diff.getSnapshotId());
return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
}
void updateBlockCollection() { void updateBlockCollection() {
if (blocks != null) { if (blocks != null) {
for(BlockInfo b : blocks) { for(BlockInfo b : blocks) {
@ -509,13 +530,13 @@ public class INodeFile extends INodeWithAdditionalFields
} }
clear(); clear();
removedINodes.add(this); removedINodes.add(this);
FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if (sf != null) { if (sf != null) {
sf.getDiffs().destroyAndCollectSnapshotBlocks(collectedBlocks);
sf.clearDiffs(); sf.clearDiffs();
} }
} }
@Override @Override
public String getName() { public String getName() {
// Get the full path name of this inode. // Get the full path name of this inode.
@ -554,39 +575,23 @@ public class INodeFile extends INodeWithAdditionalFields
@Override @Override
public final ContentSummaryComputationContext computeContentSummary( public final ContentSummaryComputationContext computeContentSummary(
final ContentSummaryComputationContext summary) { final ContentSummaryComputationContext summary) {
computeContentSummary4Snapshot(summary.getCounts()); final Content.Counts counts = summary.getCounts();
computeContentSummary4Current(summary.getCounts());
return summary;
}
private void computeContentSummary4Snapshot(final Content.Counts counts) {
// file length and diskspace only counted for the latest state of the file
// i.e. either the current state or the last snapshot
FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if (sf != null) { if (sf == null) {
counts.add(Content.LENGTH, computeFileSize());
counts.add(Content.FILE, 1);
} else {
final FileDiffList diffs = sf.getDiffs(); final FileDiffList diffs = sf.getDiffs();
final int n = diffs.asList().size(); final int n = diffs.asList().size();
counts.add(Content.FILE, n); counts.add(Content.FILE, n);
if (n > 0 && sf.isCurrentFileDeleted()) { if (n > 0 && sf.isCurrentFileDeleted()) {
counts.add(Content.LENGTH, diffs.getLast().getFileSize()); counts.add(Content.LENGTH, diffs.getLast().getFileSize());
} } else {
counts.add(Content.LENGTH, computeFileSize());
if (sf.isCurrentFileDeleted()) {
final long lastFileSize = diffs.getLast().getFileSize();
counts.add(Content.DISKSPACE, lastFileSize * getBlockReplication());
} }
} }
}
private void computeContentSummary4Current(final Content.Counts counts) {
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
if (sf != null && sf.isCurrentFileDeleted()) {
return;
}
counts.add(Content.LENGTH, computeFileSize());
counts.add(Content.FILE, 1);
counts.add(Content.DISKSPACE, diskspaceConsumed()); counts.add(Content.DISKSPACE, diskspaceConsumed());
return summary;
} }
/** The same as computeFileSize(null). */ /** The same as computeFileSize(null). */
@ -651,9 +656,36 @@ public class INodeFile extends INodeWithAdditionalFields
return size; return size;
} }
/**
* Compute size consumed by all blocks of the current file,
* including blocks in its snapshots.
* Use preferred block size for the last block if it is under construction.
*/
public final long diskspaceConsumed() { public final long diskspaceConsumed() {
// use preferred block size for the last block if it is under construction FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
return computeFileSize(true, true) * getBlockReplication(); if(sf == null) {
return computeFileSize(true, true) * getBlockReplication();
}
// Collect all distinct blocks
long size = 0;
Set<Block> allBlocks = new HashSet<Block>(Arrays.asList(getBlocks()));
List<FileDiff> diffs = sf.getDiffs().asList();
for(FileDiff diff : diffs) {
BlockInfo[] diffBlocks = diff.getBlocks();
if (diffBlocks != null) {
allBlocks.addAll(Arrays.asList(diffBlocks));
}
}
for(Block block : allBlocks) {
size += block.getNumBytes();
}
// check if the last block is under construction
BlockInfo lastBlock = getLastBlock();
if(lastBlock != null && lastBlock instanceof BlockInfoUnderConstruction) {
size += getPreferredBlockSize() - lastBlock.getNumBytes();
}
return size * getBlockReplication();
} }
public final long diskspaceConsumed(int lastSnapshotId) { public final long diskspaceConsumed(int lastSnapshotId) {
@ -706,7 +738,7 @@ public class INodeFile extends INodeWithAdditionalFields
final BlockInfo[] oldBlocks = getBlocks(); final BlockInfo[] oldBlocks = getBlocks();
if (oldBlocks == null) if (oldBlocks == null)
return 0; return 0;
//find the minimum n such that the size of the first n blocks > max // find the minimum n such that the size of the first n blocks > max
int n = 0; int n = 0;
long size = 0; long size = 0;
for(; n < oldBlocks.length && max > size; n++) { for(; n < oldBlocks.length && max > size; n++) {
@ -716,16 +748,8 @@ public class INodeFile extends INodeWithAdditionalFields
return size; return size;
// starting from block n, the data is beyond max. // starting from block n, the data is beyond max.
// resize the array. // resize the array.
final BlockInfo[] newBlocks; truncateBlocksTo(n);
if (n == 0) {
newBlocks = BlockInfo.EMPTY_ARRAY;
} else {
newBlocks = new BlockInfo[n];
System.arraycopy(oldBlocks, 0, newBlocks, 0, n);
}
// set new blocks
setBlocks(newBlocks);
// collect the blocks beyond max // collect the blocks beyond max
if (collectedBlocks != null) { if (collectedBlocks != null) {
@ -735,4 +759,67 @@ public class INodeFile extends INodeWithAdditionalFields
} }
return size; return size;
} }
void truncateBlocksTo(int n) {
final BlockInfo[] newBlocks;
if (n == 0) {
newBlocks = BlockInfo.EMPTY_ARRAY;
} else {
newBlocks = new BlockInfo[n];
System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
}
// set new blocks
setBlocks(newBlocks);
}
public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
BlocksMapUpdateInfo collectedBlocks) {
BlockInfo[] oldBlocks = getBlocks();
if(snapshotBlocks == null || oldBlocks == null)
return;
// Skip blocks in common between the file and the snapshot
int n = 0;
while(n < oldBlocks.length && n < snapshotBlocks.length &&
oldBlocks[n] == snapshotBlocks[n]) {
n++;
}
truncateBlocksTo(n);
// Collect the remaining blocks of the file
while(n < oldBlocks.length) {
collectedBlocks.addDeleteBlock(oldBlocks[n++]);
}
}
/** Exclude blocks collected for deletion that belong to a snapshot. */
void excludeSnapshotBlocks(int snapshotId,
BlocksMapUpdateInfo collectedBlocks) {
if(collectedBlocks == null || collectedBlocks.getToDeleteList().isEmpty())
return;
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if(sf == null)
return;
BlockInfo[] snapshotBlocks =
getDiffs().findEarlierSnapshotBlocks(snapshotId);
if(snapshotBlocks == null)
return;
List<Block> toDelete = collectedBlocks.getToDeleteList();
for(Block blk : snapshotBlocks) {
if(toDelete.contains(blk))
collectedBlocks.removeDeleteBlock(blk);
}
}
/**
* @return true if the block is contained in a snapshot or false otherwise.
*/
boolean isBlockInLatestSnapshot(BlockInfo block) {
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
if (sf == null || sf.getDiffs() == null)
return false;
BlockInfo[] snapshotBlocks =
getDiffs().findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
if(snapshotBlocks == null)
return false;
return Arrays.asList(snapshotBlocks).contains(block);
}
} }

View File

@ -69,8 +69,9 @@ public class NameNodeLayoutVersion {
CREATE_OVERWRITE(-58, "Use single editlog record for " + CREATE_OVERWRITE(-58, "Use single editlog record for " +
"creating file with overwrite"), "creating file with overwrite"),
XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"), XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
BLOCK_STORAGE_POLICY(-60, "Block Storage policy"); BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
TRUNCATE(-61, "Truncate");
private final FeatureInfo info; private final FeatureInfo info;
/** /**

View File

@ -163,9 +163,12 @@ abstract class AbstractINodeDiffList<N extends INode,
* id, otherwise <=. * id, otherwise <=.
* @return The id of the latest snapshot before the given snapshot. * @return The id of the latest snapshot before the given snapshot.
*/ */
private final int getPrior(int anchorId, boolean exclusive) { public final int getPrior(int anchorId, boolean exclusive) {
if (anchorId == Snapshot.CURRENT_STATE_ID) { if (anchorId == Snapshot.CURRENT_STATE_ID) {
return getLastSnapshotId(); int last = getLastSnapshotId();
if(exclusive && last == anchorId)
return Snapshot.NO_SNAPSHOT_ID;
return last;
} }
final int i = Collections.binarySearch(diffs, anchorId); final int i = Collections.binarySearch(diffs, anchorId);
if (exclusive) { // must be the one before if (exclusive) { // must be the one before
@ -290,10 +293,11 @@ abstract class AbstractINodeDiffList<N extends INode,
} }
/** Save the snapshot copy to the latest snapshot. */ /** Save the snapshot copy to the latest snapshot. */
public void saveSelf2Snapshot(int latestSnapshotId, N currentINode, public D saveSelf2Snapshot(int latestSnapshotId, N currentINode,
A snapshotCopy) throws QuotaExceededException { A snapshotCopy) throws QuotaExceededException {
D diff = null;
if (latestSnapshotId != Snapshot.CURRENT_STATE_ID) { if (latestSnapshotId != Snapshot.CURRENT_STATE_ID) {
D diff = checkAndAddLatestSnapshotDiff(latestSnapshotId, currentINode); diff = checkAndAddLatestSnapshotDiff(latestSnapshotId, currentINode);
if (diff.snapshotINode == null) { if (diff.snapshotINode == null) {
if (snapshotCopy == null) { if (snapshotCopy == null) {
snapshotCopy = createSnapshotCopy(currentINode); snapshotCopy = createSnapshotCopy(currentINode);
@ -301,6 +305,7 @@ abstract class AbstractINodeDiffList<N extends INode,
diff.saveSnapshotCopy(snapshotCopy); diff.saveSnapshotCopy(snapshotCopy);
} }
} }
return diff;
} }
@Override @Override
@ -312,4 +317,4 @@ abstract class AbstractINodeDiffList<N extends INode,
public String toString() { public String toString() {
return getClass().getSimpleName() + ": " + diffs; return getClass().getSimpleName() + ": " + diffs;
} }
} }

View File

@ -36,6 +36,10 @@ import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat; import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
import org.apache.hadoop.hdfs.server.namenode.AclFeature; import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@ -230,6 +234,20 @@ public class FSImageFormatPBSnapshot {
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
pbf.getFileSize()); pbf.getFileSize());
List<BlockProto> bpl = pbf.getBlocksList();
BlockInfo[] blocks = new BlockInfo[bpl.size()];
for(int j = 0, e = bpl.size(); j < e; ++j) {
Block blk = PBHelper.convert(bpl.get(j));
BlockInfo storedBlock = fsn.getBlockManager().getStoredBlock(blk);
if(storedBlock == null) {
storedBlock = fsn.getBlockManager().addBlockCollection(
new BlockInfo(blk, copy.getFileReplication()), file);
}
blocks[j] = storedBlock;
}
if(blocks.length > 0) {
diff.setBlocks(blocks);
}
diffs.addFirst(diff); diffs.addFirst(diff);
} }
file.addSnapshotFeature(diffs); file.addSnapshotFeature(diffs);
@ -473,6 +491,11 @@ public class FSImageFormatPBSnapshot {
SnapshotDiffSection.FileDiff.Builder fb = SnapshotDiffSection.FileDiff SnapshotDiffSection.FileDiff.Builder fb = SnapshotDiffSection.FileDiff
.newBuilder().setSnapshotId(diff.getSnapshotId()) .newBuilder().setSnapshotId(diff.getSnapshotId())
.setFileSize(diff.getFileSize()); .setFileSize(diff.getFileSize());
if(diff.getBlocks() != null) {
for(Block block : diff.getBlocks()) {
fb.addBlocks(PBHelper.convert(block));
}
}
INodeFileAttributes copy = diff.snapshotINode; INodeFileAttributes copy = diff.snapshotINode;
if (copy != null) { if (copy != null) {
fb.setName(ByteString.copyFrom(copy.getLocalNameBytes())) fb.setName(ByteString.copyFrom(copy.getLocalNameBytes()))

View File

@ -19,8 +19,10 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.io.DataOutput; import java.io.DataOutput;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.List; import java.util.List;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization; import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@ -37,10 +39,13 @@ public class FileDiff extends
/** The file size at snapshot creation time. */ /** The file size at snapshot creation time. */
private final long fileSize; private final long fileSize;
/** A copy of the INodeFile block list. Used in truncate. */
private BlockInfo[] blocks;
FileDiff(int snapshotId, INodeFile file) { FileDiff(int snapshotId, INodeFile file) {
super(snapshotId, null, null); super(snapshotId, null, null);
fileSize = file.computeFileSize(); fileSize = file.computeFileSize();
blocks = null;
} }
/** Constructor used by FSImage loading */ /** Constructor used by FSImage loading */
@ -48,20 +53,40 @@ public class FileDiff extends
FileDiff posteriorDiff, long fileSize) { FileDiff posteriorDiff, long fileSize) {
super(snapshotId, snapshotINode, posteriorDiff); super(snapshotId, snapshotINode, posteriorDiff);
this.fileSize = fileSize; this.fileSize = fileSize;
blocks = null;
} }
/** @return the file size in the snapshot. */ /** @return the file size in the snapshot. */
public long getFileSize() { public long getFileSize() {
return fileSize; return fileSize;
} }
/**
* Copy block references into the snapshot
* up to the current {@link #fileSize}.
* Should be done only once.
*/
public void setBlocks(BlockInfo[] blocks) {
if(this.blocks != null)
return;
int numBlocks = 0;
for(long s = 0; numBlocks < blocks.length && s < fileSize; numBlocks++)
s += blocks[numBlocks].getNumBytes();
this.blocks = Arrays.copyOf(blocks, numBlocks);
}
public BlockInfo[] getBlocks() {
return blocks;
}
@Override @Override
Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode, Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode,
FileDiff posterior, BlocksMapUpdateInfo collectedBlocks, FileDiff posterior, BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes) {
return currentINode.getFileWithSnapshotFeature() FileWithSnapshotFeature sf = currentINode.getFileWithSnapshotFeature();
.updateQuotaAndCollectBlocks(currentINode, posterior, collectedBlocks, assert sf != null : "FileWithSnapshotFeature is null";
removedINodes); return sf.updateQuotaAndCollectBlocks(
currentINode, posterior, collectedBlocks, removedINodes);
} }
@Override @Override
@ -91,4 +116,13 @@ public class FileDiff extends
.updateQuotaAndCollectBlocks(currentINode, this, collectedBlocks, .updateQuotaAndCollectBlocks(currentINode, this, collectedBlocks,
removedINodes); removedINodes);
} }
public void destroyAndCollectSnapshotBlocks(
BlocksMapUpdateInfo collectedBlocks) {
if(blocks == null || collectedBlocks == null)
return;
for(BlockInfo blk : blocks)
collectedBlocks.addDeleteBlock(blk);
blocks = null;
}
} }

View File

@ -17,6 +17,13 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode.snapshot; package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes; import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
@ -33,4 +40,95 @@ public class FileDiffList extends
INodeFileAttributes createSnapshotCopy(INodeFile currentINode) { INodeFileAttributes createSnapshotCopy(INodeFile currentINode) {
return new INodeFileAttributes.SnapshotCopy(currentINode); return new INodeFileAttributes.SnapshotCopy(currentINode);
} }
public void destroyAndCollectSnapshotBlocks(
BlocksMapUpdateInfo collectedBlocks) {
for(FileDiff d : asList())
d.destroyAndCollectSnapshotBlocks(collectedBlocks);
}
public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile,
INodeFileAttributes snapshotCopy, boolean withBlocks)
throws QuotaExceededException {
final FileDiff diff =
super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
if(withBlocks) // Store blocks if this is the first update
diff.setBlocks(iNodeFile.getBlocks());
}
public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if(snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfo[] blocks = null;
for(i = i >= 0 ? i : -i; i < diffs.size(); i--) {
blocks = diffs.get(i).getBlocks();
if(blocks != null) {
break;
}
}
return blocks;
}
public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if(snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfo[] blocks = null;
for(i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
blocks = diffs.get(i).getBlocks();
if(blocks != null) {
break;
}
}
return blocks;
}
/**
* Copy blocks from the removed snapshot into the previous snapshot
* up to the file length of the latter.
* Collect unused blocks of the removed snapshot.
*/
void combineAndCollectSnapshotBlocks(INodeFile file,
FileDiff removed,
BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes) {
BlockInfo[] removedBlocks = removed.getBlocks();
if(removedBlocks == null) {
FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
assert sf != null : "FileWithSnapshotFeature is null";
if(sf.isCurrentFileDeleted())
sf.collectBlocksAndClear(file, collectedBlocks, removedINodes);
return;
}
int p = getPrior(removed.getSnapshotId(), true);
FileDiff earlierDiff = p == Snapshot.NO_SNAPSHOT_ID ? null : getDiffById(p);
// Copy blocks to the previous snapshot if not set already
if(earlierDiff != null)
earlierDiff.setBlocks(removedBlocks);
BlockInfo[] earlierBlocks =
(earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks());
// Find later snapshot (or file itself) with blocks
BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
// Skip blocks, which belong to either the earlier or the later lists
int i = 0;
for(; i < removedBlocks.length; i++) {
if(i < earlierBlocks.length && removedBlocks[i] == earlierBlocks[i])
continue;
if(i < laterBlocks.length && removedBlocks[i] == laterBlocks[i])
continue;
break;
}
// Collect the remaining blocks of the file
while(i < removedBlocks.length) {
collectedBlocks.addDeleteBlock(removedBlocks[i++]);
}
}
} }

View File

@ -21,6 +21,7 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.namenode.AclFeature; import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@ -154,18 +155,19 @@ public class FileWithSnapshotFeature implements INode.Feature {
AclStorage.removeAclFeature(aclFeature); AclStorage.removeAclFeature(aclFeature);
} }
} }
collectBlocksAndClear(file, collectedBlocks, removedINodes); getDiffs().combineAndCollectSnapshotBlocks(
file, removed, collectedBlocks, removedINodes);
long dsDelta = oldDiskspace - file.diskspaceConsumed(); long dsDelta = oldDiskspace - file.diskspaceConsumed();
return Quota.Counts.newInstance(0, dsDelta); return Quota.Counts.newInstance(0, dsDelta);
} }
/** /**
* If some blocks at the end of the block list no longer belongs to * If some blocks at the end of the block list no longer belongs to
* any inode, collect them and update the block list. * any inode, collect them and update the block list.
*/ */
private void collectBlocksAndClear(final INodeFile file, public void collectBlocksAndClear(final INodeFile file,
final BlocksMapUpdateInfo info, final List<INode> removedINodes) { final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
// check if everything is deleted. // check if everything is deleted.
if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) { if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
@ -174,13 +176,19 @@ public class FileWithSnapshotFeature implements INode.Feature {
} }
// find max file size. // find max file size.
final long max; final long max;
FileDiff diff = getDiffs().getLast();
if (isCurrentFileDeleted()) { if (isCurrentFileDeleted()) {
final FileDiff last = getDiffs().getLast(); max = diff == null? 0: diff.getFileSize();
max = last == null? 0: last.getFileSize();
} else { } else {
max = file.computeFileSize(); max = file.computeFileSize();
} }
file.collectBlocksBeyondMax(max, info); // Collect blocks that should be deleted
FileDiff last = diffs.getLast();
BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
if(snapshotBlocks == null)
file.collectBlocksBeyondMax(max, info);
else
file.collectBlocksBeyondSnapshot(snapshotBlocks, info);
} }
} }

View File

@ -22,6 +22,7 @@ import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -53,8 +54,8 @@ public class BlockRecoveryCommand extends DatanodeCommand {
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public static class RecoveringBlock extends LocatedBlock { public static class RecoveringBlock extends LocatedBlock {
private boolean truncate;
private final long newGenerationStamp; private final long newGenerationStamp;
private final Block recoveryBlock;
/** /**
* Create RecoveringBlock. * Create RecoveringBlock.
@ -62,15 +63,17 @@ public class BlockRecoveryCommand extends DatanodeCommand {
public RecoveringBlock(ExtendedBlock b, DatanodeInfo[] locs, long newGS) { public RecoveringBlock(ExtendedBlock b, DatanodeInfo[] locs, long newGS) {
super(b, locs, -1, false); // startOffset is unknown super(b, locs, -1, false); // startOffset is unknown
this.newGenerationStamp = newGS; this.newGenerationStamp = newGS;
this.recoveryBlock = null;
} }
/** /**
* RecoveryingBlock with truncate option. * Create RecoveringBlock with copy-on-truncate option.
*/ */
public RecoveringBlock(ExtendedBlock b, DatanodeInfo[] locs, long newGS, public RecoveringBlock(ExtendedBlock b, DatanodeInfo[] locs,
boolean truncate) { Block recoveryBlock) {
this(b, locs, newGS); super(b, locs, -1, false); // startOffset is unknown
this.truncate = truncate; this.newGenerationStamp = recoveryBlock.getGenerationStamp();
this.recoveryBlock = recoveryBlock;
} }
/** /**
@ -82,10 +85,10 @@ public class BlockRecoveryCommand extends DatanodeCommand {
} }
/** /**
* Return whether to truncate the block to the ExtendedBlock's length. * Return the new block.
*/ */
public boolean getTruncateFlag() { public Block getNewBlock() {
return truncate; return recoveryBlock;
} }
} }

View File

@ -67,5 +67,6 @@ public interface InterDatanodeProtocol {
* Update replica with the new generation stamp and length. * Update replica with the new generation stamp and length.
*/ */
String updateReplicaUnderRecovery(ExtendedBlock oldBlock, long recoveryId, String updateReplicaUnderRecovery(ExtendedBlock oldBlock, long recoveryId,
long newLength) throws IOException; long newBlockId, long newLength)
throws IOException;
} }

View File

@ -59,6 +59,8 @@ message UpdateReplicaUnderRecoveryRequestProto {
required ExtendedBlockProto block = 1; // Block identifier required ExtendedBlockProto block = 1; // Block identifier
required uint64 recoveryId = 2; // New genstamp of the replica required uint64 recoveryId = 2; // New genstamp of the replica
required uint64 newLength = 3; // New length of the replica required uint64 newLength = 3; // New length of the replica
// New blockId for copy (truncate), default is 0.
optional uint64 newBlockId = 4 [default = 0];
} }
/** /**

View File

@ -270,6 +270,7 @@ message SnapshotDiffSection {
optional uint64 fileSize = 2; optional uint64 fileSize = 2;
optional bytes name = 3; optional bytes name = 3;
optional INodeSection.INodeFile snapshotCopy = 4; optional INodeSection.INodeFile snapshotCopy = 4;
repeated BlockProto blocks = 5;
} }
message DiffEntry { message DiffEntry {

View File

@ -554,9 +554,9 @@ enum ReplicaStateProto {
* Block that needs to be recovered with at a given location * Block that needs to be recovered with at a given location
*/ */
message RecoveringBlockProto { message RecoveringBlockProto {
required uint64 newGenStamp = 1; // New genstamp post recovery required uint64 newGenStamp = 1; // New genstamp post recovery
required LocatedBlockProto block = 2; // Block to be recovered required LocatedBlockProto block = 2; // Block to be recovered
optional bool truncateFlag = 3; // Block needs to be truncated optional BlockProto truncateBlock = 3; // New block for recovery (truncate)
} }
/** /**

View File

@ -1113,6 +1113,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
@Override // FsDatasetSpi @Override // FsDatasetSpi
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock, public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long recoveryId,
long newBlockId,
long newlength) { long newlength) {
// Caller does not care about the exact Storage UUID returned. // Caller does not care about the exact Storage UUID returned.
return datanodeUuid; return datanodeUuid;

View File

@ -56,7 +56,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -216,10 +215,10 @@ public class TestBlockRecovery {
syncList.add(record1); syncList.add(record1);
syncList.add(record2); syncList.add(record2);
when(dn1.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(), when(dn1.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(),
anyLong())).thenReturn("storage1"); anyLong(), anyLong())).thenReturn("storage1");
when(dn2.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(), when(dn2.updateReplicaUnderRecovery((ExtendedBlock)anyObject(), anyLong(),
anyLong())).thenReturn("storage2"); anyLong(), anyLong())).thenReturn("storage2");
dn.syncBlock(rBlock, syncList); dn.syncBlock(rBlock, syncList);
} }
@ -242,8 +241,10 @@ public class TestBlockRecovery {
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class); InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); REPLICA_LEN1);
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
REPLICA_LEN1);
// two finalized replicas have different length // two finalized replicas have different length
replica1 = new ReplicaRecoveryInfo(BLOCK_ID, replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
@ -281,8 +282,10 @@ public class TestBlockRecovery {
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class); InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); REPLICA_LEN1);
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
REPLICA_LEN1);
// rbw replica has a different length from the finalized one // rbw replica has a different length from the finalized one
replica1 = new ReplicaRecoveryInfo(BLOCK_ID, replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
@ -294,9 +297,9 @@ public class TestBlockRecovery {
dn2 = mock(InterDatanodeProtocol.class); dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
verify(dn2, never()).updateReplicaUnderRecovery( verify(dn2, never()).updateReplicaUnderRecovery(
block, RECOVERY_ID, REPLICA_LEN1); block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
} }
/** /**
@ -320,9 +323,10 @@ public class TestBlockRecovery {
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class); InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
REPLICA_LEN1);
verify(dn2, never()).updateReplicaUnderRecovery( verify(dn2, never()).updateReplicaUnderRecovery(
block, RECOVERY_ID, REPLICA_LEN1); block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
// rbw replica has a different length from the finalized one // rbw replica has a different length from the finalized one
replica1 = new ReplicaRecoveryInfo(BLOCK_ID, replica1 = new ReplicaRecoveryInfo(BLOCK_ID,
@ -334,9 +338,10 @@ public class TestBlockRecovery {
dn2 = mock(InterDatanodeProtocol.class); dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
REPLICA_LEN1);
verify(dn2, never()).updateReplicaUnderRecovery( verify(dn2, never()).updateReplicaUnderRecovery(
block, RECOVERY_ID, REPLICA_LEN1); block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
} }
/** /**
@ -359,8 +364,8 @@ public class TestBlockRecovery {
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2); long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
testSyncReplicas(replica1, replica2, dn1, dn2, minLen); testSyncReplicas(replica1, replica2, dn1, dn2, minLen);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen); verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
} }
/** /**
@ -382,9 +387,10 @@ public class TestBlockRecovery {
InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class); InterDatanodeProtocol dn2 = mock(InterDatanodeProtocol.class);
testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1); testSyncReplicas(replica1, replica2, dn1, dn2, REPLICA_LEN1);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, REPLICA_LEN1); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID,
REPLICA_LEN1);
verify(dn2, never()).updateReplicaUnderRecovery( verify(dn2, never()).updateReplicaUnderRecovery(
block, RECOVERY_ID, REPLICA_LEN1); block, RECOVERY_ID, BLOCK_ID, REPLICA_LEN1);
} }
/** /**
@ -408,8 +414,8 @@ public class TestBlockRecovery {
long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2); long minLen = Math.min(REPLICA_LEN1, REPLICA_LEN2);
testSyncReplicas(replica1, replica2, dn1, dn2, minLen); testSyncReplicas(replica1, replica2, dn1, dn2, minLen);
verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen); verify(dn1).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, minLen); verify(dn2).updateReplicaUnderRecovery(block, RECOVERY_ID, BLOCK_ID, minLen);
} }
private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException { private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
@ -510,7 +516,7 @@ public class TestBlockRecovery {
} }
DataNode spyDN = spy(dn); DataNode spyDN = spy(dn);
doThrow(new IOException()).when(spyDN).updateReplicaUnderRecovery( doThrow(new IOException()).when(spyDN).updateReplicaUnderRecovery(
block, RECOVERY_ID, block.getNumBytes()); block, RECOVERY_ID, BLOCK_ID, block.getNumBytes());
try { try {
spyDN.syncBlock(rBlock, initBlockRecords(spyDN)); spyDN.syncBlock(rBlock, initBlockRecords(spyDN));
fail("Sync should fail"); fail("Sync should fail");
@ -631,7 +637,8 @@ public class TestBlockRecovery {
recoveryInitResult.get()); recoveryInitResult.get());
dataNode.updateReplicaUnderRecovery(block.getBlock(), block.getBlock() dataNode.updateReplicaUnderRecovery(block.getBlock(), block.getBlock()
.getGenerationStamp() + 1, block.getBlockSize()); .getGenerationStamp() + 1, block.getBlock().getBlockId(),
block.getBlockSize());
} finally { } finally {
if (null != cluster) { if (null != cluster) {
cluster.shutdown(); cluster.shutdown();

View File

@ -198,7 +198,8 @@ public class TestInterDatanodeProtocol {
//verify updateBlock //verify updateBlock
ExtendedBlock newblock = new ExtendedBlock(b.getBlockPoolId(), ExtendedBlock newblock = new ExtendedBlock(b.getBlockPoolId(),
b.getBlockId(), b.getNumBytes()/2, b.getGenerationStamp()+1); b.getBlockId(), b.getNumBytes()/2, b.getGenerationStamp()+1);
idp.updateReplicaUnderRecovery(b, recoveryId, newblock.getNumBytes()); idp.updateReplicaUnderRecovery(b, recoveryId, b.getBlockId(),
newblock.getNumBytes());
checkMetaInfo(newblock, datanode); checkMetaInfo(newblock, datanode);
// Verify correct null response trying to init recovery for a missing block // Verify correct null response trying to init recovery for a missing block
@ -368,7 +369,8 @@ public class TestInterDatanodeProtocol {
.getBlockId(), rri.getNumBytes() - 1, rri.getGenerationStamp()); .getBlockId(), rri.getNumBytes() - 1, rri.getGenerationStamp());
try { try {
//update should fail //update should fail
fsdataset.updateReplicaUnderRecovery(tmp, recoveryid, newlength); fsdataset.updateReplicaUnderRecovery(tmp, recoveryid,
tmp.getBlockId(), newlength);
Assert.fail(); Assert.fail();
} catch(IOException ioe) { } catch(IOException ioe) {
System.out.println("GOOD: getting " + ioe); System.out.println("GOOD: getting " + ioe);
@ -377,7 +379,8 @@ public class TestInterDatanodeProtocol {
//update //update
final String storageID = fsdataset.updateReplicaUnderRecovery( final String storageID = fsdataset.updateReplicaUnderRecovery(
new ExtendedBlock(b.getBlockPoolId(), rri), recoveryid, newlength); new ExtendedBlock(b.getBlockPoolId(), rri), recoveryid,
rri.getBlockId(), newlength);
assertTrue(storageID != null); assertTrue(storageID != null);
} finally { } finally {

View File

@ -71,6 +71,7 @@ public class TestCommitBlockSynchronization {
doReturn(true).when(file).isUnderConstruction(); doReturn(true).when(file).isUnderConstruction();
doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class)); doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class));
doReturn(blockInfo).when(file).getLastBlock();
doReturn("").when(namesystemSpy).closeFileCommitBlocks( doReturn("").when(namesystemSpy).closeFileCommitBlocks(
any(INodeFile.class), any(BlockInfo.class)); any(INodeFile.class), any(BlockInfo.class));
doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog(); doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog();
@ -105,6 +106,7 @@ public class TestCommitBlockSynchronization {
completedBlockInfo.setGenerationStamp(genStamp); completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy) doReturn(completedBlockInfo).when(namesystemSpy)
.getStoredBlock(any(Block.class)); .getStoredBlock(any(Block.class));
doReturn(completedBlockInfo).when(file).getLastBlock();
// Repeat the call to make sure it does not throw // Repeat the call to make sure it does not throw
namesystemSpy.commitBlockSynchronization( namesystemSpy.commitBlockSynchronization(
@ -176,6 +178,7 @@ public class TestCommitBlockSynchronization {
completedBlockInfo.setGenerationStamp(genStamp); completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy) doReturn(completedBlockInfo).when(namesystemSpy)
.getStoredBlock(any(Block.class)); .getStoredBlock(any(Block.class));
doReturn(completedBlockInfo).when(file).getLastBlock();
namesystemSpy.commitBlockSynchronization( namesystemSpy.commitBlockSynchronization(
lastBlock, genStamp, length, true, false, newTargets, null); lastBlock, genStamp, length, true, false, newTargets, null);

View File

@ -18,14 +18,22 @@
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.IOException; import java.io.IOException;
import java.net.InetAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
@ -39,14 +47,14 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -57,6 +65,7 @@ public class TestFileTruncate {
GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.ALL); GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.ALL);
GenericTestUtils.setLogLevel(FSEditLogLoader.LOG, Level.ALL); GenericTestUtils.setLogLevel(FSEditLogLoader.LOG, Level.ALL);
} }
static final Log LOG = LogFactory.getLog(TestFileTruncate.class);
static final int BLOCK_SIZE = 4; static final int BLOCK_SIZE = 4;
static final short REPLICATION = 3; static final short REPLICATION = 3;
static final int DATANODE_NUM = 3; static final int DATANODE_NUM = 3;
@ -129,6 +138,287 @@ public class TestFileTruncate {
fs.delete(parent, true); fs.delete(parent, true);
} }
@Test
public void testSnapshotWithAppendTruncate() throws IOException {
testSnapshotWithAppendTruncate(0, 1, 2);
testSnapshotWithAppendTruncate(0, 2, 1);
testSnapshotWithAppendTruncate(1, 0, 2);
testSnapshotWithAppendTruncate(1, 2, 0);
testSnapshotWithAppendTruncate(2, 0, 1);
testSnapshotWithAppendTruncate(2, 1, 0);
}
/**
* Create three snapshots with appended and truncated file.
* Delete snapshots in the specified order and verify that
* remaining snapshots are still readable.
*/
void testSnapshotWithAppendTruncate(int ... deleteOrder) throws IOException {
FSDirectory fsDir = cluster.getNamesystem().getFSDirectory();
Path parent = new Path("/test");
fs.mkdirs(parent);
fs.setQuota(parent, 100, 1000);
fs.allowSnapshot(parent);
String truncateFile = "testSnapshotWithAppendTruncate";
final Path src = new Path(parent, truncateFile);
int[] length = new int[4];
length[0] = 2 * BLOCK_SIZE + BLOCK_SIZE / 2;
DFSTestUtil.createFile(fs, src, 64, length[0], BLOCK_SIZE, REPLICATION, 0L);
Block firstBlk = getLocatedBlocks(src).get(0).getBlock().getLocalBlock();
Path[] snapshotFiles = new Path[4];
// Diskspace consumed should be 10 bytes * 3. [blk 1,2,3]
ContentSummary contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(30L));
// Add file to snapshot and append
String[] ss = new String[] {"ss0", "ss1", "ss2", "ss3"};
Path snapshotDir = fs.createSnapshot(parent, ss[0]);
snapshotFiles[0] = new Path(snapshotDir, truncateFile);
length[1] = length[2] = length[0] + BLOCK_SIZE + 1;
DFSTestUtil.appendFile(fs, src, BLOCK_SIZE + 1);
Block lastBlk = getLocatedBlocks(src).getLastLocatedBlock()
.getBlock().getLocalBlock();
// Diskspace consumed should be 15 bytes * 3. [blk 1,2,3,4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(45L));
// Create another snapshot without changes
snapshotDir = fs.createSnapshot(parent, ss[1]);
snapshotFiles[1] = new Path(snapshotDir, truncateFile);
// Create another snapshot and append
snapshotDir = fs.createSnapshot(parent, ss[2]);
snapshotFiles[2] = new Path(snapshotDir, truncateFile);
DFSTestUtil.appendFile(fs, src, BLOCK_SIZE -1 + BLOCK_SIZE / 2);
Block appendedBlk = getLocatedBlocks(src).getLastLocatedBlock()
.getBlock().getLocalBlock();
// Diskspace consumed should be 20 bytes * 3. [blk 1,2,3,4,5]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(60L));
// Truncate to block boundary
int newLength = length[0] + BLOCK_SIZE / 2;
boolean isReady = fs.truncate(src, newLength);
assertTrue("Recovery is not expected.", isReady);
assertFileLength(snapshotFiles[2], length[2]);
assertFileLength(snapshotFiles[1], length[1]);
assertFileLength(snapshotFiles[0], length[0]);
assertBlockNotPresent(appendedBlk);
// Diskspace consumed should be 16 bytes * 3. [blk 1,2,3 SS:4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(48L));
// Truncate full block again
newLength = length[0] - BLOCK_SIZE / 2;
isReady = fs.truncate(src, newLength);
assertTrue("Recovery is not expected.", isReady);
assertFileLength(snapshotFiles[2], length[2]);
assertFileLength(snapshotFiles[1], length[1]);
assertFileLength(snapshotFiles[0], length[0]);
// Diskspace consumed should be 16 bytes * 3. [blk 1,2 SS:3,4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(48L));
// Truncate half of the last block
newLength -= BLOCK_SIZE / 2;
isReady = fs.truncate(src, newLength);
assertFalse("Recovery is expected.", isReady);
checkBlockRecovery(src);
assertFileLength(snapshotFiles[2], length[2]);
assertFileLength(snapshotFiles[1], length[1]);
assertFileLength(snapshotFiles[0], length[0]);
Block replacedBlk = getLocatedBlocks(src).getLastLocatedBlock()
.getBlock().getLocalBlock();
// Diskspace consumed should be 16 bytes * 3. [blk 1,6 SS:2,3,4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(54L));
snapshotDir = fs.createSnapshot(parent, ss[3]);
snapshotFiles[3] = new Path(snapshotDir, truncateFile);
length[3] = newLength;
// Delete file. Should still be able to read snapshots
int numINodes = fsDir.getInodeMapSize();
isReady = fs.delete(src, false);
assertTrue("Delete failed.", isReady);
assertFileLength(snapshotFiles[3], length[3]);
assertFileLength(snapshotFiles[2], length[2]);
assertFileLength(snapshotFiles[1], length[1]);
assertFileLength(snapshotFiles[0], length[0]);
assertEquals("Number of INodes should not change",
numINodes, fsDir.getInodeMapSize());
fs.deleteSnapshot(parent, ss[3]);
assertBlockExists(firstBlk);
assertBlockExists(lastBlk);
assertBlockNotPresent(replacedBlk);
// Diskspace consumed should be 16 bytes * 3. [SS:1,2,3,4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(48L));
// delete snapshots in the specified order
fs.deleteSnapshot(parent, ss[deleteOrder[0]]);
assertFileLength(snapshotFiles[deleteOrder[1]], length[deleteOrder[1]]);
assertFileLength(snapshotFiles[deleteOrder[2]], length[deleteOrder[2]]);
assertBlockExists(firstBlk);
assertBlockExists(lastBlk);
assertEquals("Number of INodes should not change",
numINodes, fsDir.getInodeMapSize());
// Diskspace consumed should be 16 bytes * 3. [SS:1,2,3,4]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(48L));
fs.deleteSnapshot(parent, ss[deleteOrder[1]]);
assertFileLength(snapshotFiles[deleteOrder[2]], length[deleteOrder[2]]);
assertBlockExists(firstBlk);
contentSummary = fs.getContentSummary(parent);
if(fs.exists(snapshotFiles[0])) {
// Diskspace consumed should be 0 bytes * 3. [SS:1,2,3]
assertBlockNotPresent(lastBlk);
assertThat(contentSummary.getSpaceConsumed(), is(36L));
} else {
// Diskspace consumed should be 48 bytes * 3. [SS:1,2,3,4]
assertThat(contentSummary.getSpaceConsumed(), is(48L));
}
assertEquals("Number of INodes should not change",
numINodes, fsDir .getInodeMapSize());
fs.deleteSnapshot(parent, ss[deleteOrder[2]]);
assertBlockNotPresent(firstBlk);
assertBlockNotPresent(lastBlk);
// Diskspace consumed should be 0 bytes * 3. []
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(0L));
assertNotEquals("Number of INodes should change",
numINodes, fsDir.getInodeMapSize());
}
/**
* Create three snapshots with file truncated 3 times.
* Delete snapshots in the specified order and verify that
* remaining snapshots are still readable.
*/
@Test
public void testSnapshotWithTruncates() throws IOException {
testSnapshotWithTruncates(0, 1, 2);
testSnapshotWithTruncates(0, 2, 1);
testSnapshotWithTruncates(1, 0, 2);
testSnapshotWithTruncates(1, 2, 0);
testSnapshotWithTruncates(2, 0, 1);
testSnapshotWithTruncates(2, 1, 0);
}
void testSnapshotWithTruncates(int ... deleteOrder) throws IOException {
Path parent = new Path("/test");
fs.mkdirs(parent);
fs.setQuota(parent, 100, 1000);
fs.allowSnapshot(parent);
String truncateFile = "testSnapshotWithTruncates";
final Path src = new Path(parent, truncateFile);
int[] length = new int[3];
length[0] = 3 * BLOCK_SIZE;
DFSTestUtil.createFile(fs, src, 64, length[0], BLOCK_SIZE, REPLICATION, 0L);
Block firstBlk = getLocatedBlocks(src).get(0).getBlock().getLocalBlock();
Block lastBlk = getLocatedBlocks(src).getLastLocatedBlock()
.getBlock().getLocalBlock();
Path[] snapshotFiles = new Path[3];
// Diskspace consumed should be 12 bytes * 3. [blk 1,2,3]
ContentSummary contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(36L));
// Add file to snapshot and append
String[] ss = new String[] {"ss0", "ss1", "ss2"};
Path snapshotDir = fs.createSnapshot(parent, ss[0]);
snapshotFiles[0] = new Path(snapshotDir, truncateFile);
length[1] = 2 * BLOCK_SIZE;
boolean isReady = fs.truncate(src, 2 * BLOCK_SIZE);
assertTrue("Recovery is not expected.", isReady);
// Diskspace consumed should be 12 bytes * 3. [blk 1,2 SS:3]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(36L));
snapshotDir = fs.createSnapshot(parent, ss[1]);
snapshotFiles[1] = new Path(snapshotDir, truncateFile);
// Create another snapshot with truncate
length[2] = BLOCK_SIZE + BLOCK_SIZE / 2;
isReady = fs.truncate(src, BLOCK_SIZE + BLOCK_SIZE / 2);
assertFalse("Recovery is expected.", isReady);
checkBlockRecovery(src);
snapshotDir = fs.createSnapshot(parent, ss[2]);
snapshotFiles[2] = new Path(snapshotDir, truncateFile);
assertFileLength(snapshotFiles[0], length[0]);
assertBlockExists(lastBlk);
// Diskspace consumed should be 14 bytes * 3. [blk 1,4 SS:2,3]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(42L));
fs.deleteSnapshot(parent, ss[deleteOrder[0]]);
assertFileLength(snapshotFiles[deleteOrder[1]], length[deleteOrder[1]]);
assertFileLength(snapshotFiles[deleteOrder[2]], length[deleteOrder[2]]);
assertFileLength(src, length[2]);
assertBlockExists(firstBlk);
contentSummary = fs.getContentSummary(parent);
if(fs.exists(snapshotFiles[0])) {
// Diskspace consumed should be 14 bytes * 3. [blk 1,4 SS:2,3]
assertThat(contentSummary.getSpaceConsumed(), is(42L));
assertBlockExists(lastBlk);
} else {
// Diskspace consumed should be 10 bytes * 3. [blk 1,4 SS:2]
assertThat(contentSummary.getSpaceConsumed(), is(30L));
assertBlockNotPresent(lastBlk);
}
fs.deleteSnapshot(parent, ss[deleteOrder[1]]);
assertFileLength(snapshotFiles[deleteOrder[2]], length[deleteOrder[2]]);
assertFileLength(src, length[2]);
assertBlockExists(firstBlk);
contentSummary = fs.getContentSummary(parent);
if(fs.exists(snapshotFiles[0])) {
// Diskspace consumed should be 14 bytes * 3. [blk 1,4 SS:2,3]
assertThat(contentSummary.getSpaceConsumed(), is(42L));
assertBlockExists(lastBlk);
} else if(fs.exists(snapshotFiles[1])) {
// Diskspace consumed should be 10 bytes * 3. [blk 1,4 SS:2]
assertThat(contentSummary.getSpaceConsumed(), is(30L));
assertBlockNotPresent(lastBlk);
} else {
// Diskspace consumed should be 6 bytes * 3. [blk 1,4 SS:]
assertThat(contentSummary.getSpaceConsumed(), is(18L));
assertBlockNotPresent(lastBlk);
}
fs.deleteSnapshot(parent, ss[deleteOrder[2]]);
assertFileLength(src, length[2]);
assertBlockExists(firstBlk);
// Diskspace consumed should be 6 bytes * 3. [blk 1,4 SS:]
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(18L));
assertThat(contentSummary.getLength(), is(6L));
fs.delete(src, false);
assertBlockNotPresent(firstBlk);
// Diskspace consumed should be 0 bytes * 3. []
contentSummary = fs.getContentSummary(parent);
assertThat(contentSummary.getSpaceConsumed(), is(0L));
}
/** /**
* Failure / recovery test for truncate. * Failure / recovery test for truncate.
* In this failure the DNs fail to recover the blocks and the NN triggers * In this failure the DNs fail to recover the blocks and the NN triggers
@ -159,8 +449,6 @@ public class TestFileTruncate {
boolean isReady = fs.truncate(p, newLength); boolean isReady = fs.truncate(p, newLength);
assertThat("truncate should have triggered block recovery.", assertThat("truncate should have triggered block recovery.",
isReady, is(false)); isReady, is(false));
FileStatus fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLength));
boolean recoveryTriggered = false; boolean recoveryTriggered = false;
for(int i = 0; i < RECOVERY_ATTEMPTS; i++) { for(int i = 0; i < RECOVERY_ATTEMPTS; i++) {
@ -168,8 +456,6 @@ public class TestFileTruncate {
NameNodeAdapter.getLeaseHolderForPath(cluster.getNameNode(), NameNodeAdapter.getLeaseHolderForPath(cluster.getNameNode(),
p.toUri().getPath()); p.toUri().getPath());
if(leaseHolder.equals(HdfsServerConstants.NAMENODE_LEASE_HOLDER)) { if(leaseHolder.equals(HdfsServerConstants.NAMENODE_LEASE_HOLDER)) {
cluster.startDataNodes(conf, DATANODE_NUM, true,
HdfsServerConstants.StartupOption.REGULAR, null);
recoveryTriggered = true; recoveryTriggered = true;
break; break;
} }
@ -177,6 +463,9 @@ public class TestFileTruncate {
} }
assertThat("lease recovery should have occurred in ~" + assertThat("lease recovery should have occurred in ~" +
SLEEP * RECOVERY_ATTEMPTS + " ms.", recoveryTriggered, is(true)); SLEEP * RECOVERY_ATTEMPTS + " ms.", recoveryTriggered, is(true));
cluster.startDataNodes(conf, DATANODE_NUM, true,
StartupOption.REGULAR, null);
cluster.waitActive();
checkBlockRecovery(p); checkBlockRecovery(p);
@ -184,10 +473,10 @@ public class TestFileTruncate {
.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD, .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
HdfsConstants.LEASE_HARDLIMIT_PERIOD); HdfsConstants.LEASE_HARDLIMIT_PERIOD);
fileStatus = fs.getFileStatus(p); FileStatus fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLength)); assertThat(fileStatus.getLen(), is((long) newLength));
AppendTestUtil.checkFullFile(fs, p, newLength, contents, p.toString()); checkFullFile(p, newLength, contents);
fs.delete(p, false); fs.delete(p, false);
} }
@ -198,10 +487,9 @@ public class TestFileTruncate {
public void testTruncateEditLogLoad() throws IOException { public void testTruncateEditLogLoad() throws IOException {
int startingFileSize = 2 * BLOCK_SIZE + BLOCK_SIZE / 2; int startingFileSize = 2 * BLOCK_SIZE + BLOCK_SIZE / 2;
int toTruncate = 1; int toTruncate = 1;
final String s = "/testTruncateEditLogLoad";
final Path p = new Path(s);
byte[] contents = AppendTestUtil.initBuffer(startingFileSize); byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
final Path p = new Path("/testTruncateEditLogLoad");
writeContents(contents, startingFileSize, p); writeContents(contents, startingFileSize, p);
int newLength = startingFileSize - toTruncate; int newLength = startingFileSize - toTruncate;
@ -209,54 +497,183 @@ public class TestFileTruncate {
assertThat("truncate should have triggered block recovery.", assertThat("truncate should have triggered block recovery.",
isReady, is(false)); isReady, is(false));
checkBlockRecovery(p);
cluster.restartNameNode(); cluster.restartNameNode();
String holder = UserGroupInformation.getCurrentUser().getUserName();
cluster.getNamesystem().recoverLease(s, holder, "");
checkBlockRecovery(p);
FileStatus fileStatus = fs.getFileStatus(p); FileStatus fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLength)); assertThat(fileStatus.getLen(), is((long) newLength));
AppendTestUtil.checkFullFile(fs, p, newLength, contents, p.toString()); checkFullFile(p, newLength, contents);
fs.delete(p, false); fs.delete(p, false);
} }
/**
* Upgrade, RollBack, and restart test for Truncate.
*/
@Test
public void testUpgradeAndRestart() throws IOException {
Path parent = new Path("/test");
fs.mkdirs(parent);
fs.setQuota(parent, 100, 1000);
fs.allowSnapshot(parent);
String truncateFile = "testUpgrade";
final Path p = new Path(parent, truncateFile);
int startingFileSize = 2 * BLOCK_SIZE;
int toTruncate = 1;
byte[] contents = AppendTestUtil.initBuffer(startingFileSize);
writeContents(contents, startingFileSize, p);
Path snapshotDir = fs.createSnapshot(parent, "ss0");
Path snapshotFile = new Path(snapshotDir, truncateFile);
int newLengthBeforeUpgrade = startingFileSize - toTruncate;
boolean isReady = fs.truncate(p, newLengthBeforeUpgrade);
assertThat("truncate should have triggered block recovery.",
isReady, is(false));
checkBlockRecovery(p);
checkFullFile(p, newLengthBeforeUpgrade, contents);
assertFileLength(snapshotFile, startingFileSize);
long totalBlockBefore = cluster.getNamesystem().getBlocksTotal();
restartCluster(StartupOption.UPGRADE);
assertThat("SafeMode should be OFF",
cluster.getNamesystem().isInSafeMode(), is(false));
assertThat("NameNode should be performing upgrade.",
cluster.getNamesystem().isUpgradeFinalized(), is(false));
FileStatus fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLengthBeforeUpgrade));
int newLengthAfterUpgrade = newLengthBeforeUpgrade - toTruncate;
Block oldBlk = getLocatedBlocks(p).getLastLocatedBlock()
.getBlock().getLocalBlock();
isReady = fs.truncate(p, newLengthAfterUpgrade);
assertThat("truncate should have triggered block recovery.",
isReady, is(false));
fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLengthAfterUpgrade));
assertThat("Should copy on truncate during upgrade",
getLocatedBlocks(p).getLastLocatedBlock().getBlock()
.getLocalBlock().getBlockId(), is(not(equalTo(oldBlk.getBlockId()))));
checkBlockRecovery(p);
checkFullFile(p, newLengthAfterUpgrade, contents);
assertThat("Total block count should be unchanged from copy-on-truncate",
cluster.getNamesystem().getBlocksTotal(), is(totalBlockBefore));
restartCluster(StartupOption.ROLLBACK);
assertThat("File does not exist " + p, fs.exists(p), is(true));
fileStatus = fs.getFileStatus(p);
assertThat(fileStatus.getLen(), is((long) newLengthBeforeUpgrade));
checkFullFile(p, newLengthBeforeUpgrade, contents);
assertThat("Total block count should be unchanged from rolling back",
cluster.getNamesystem().getBlocksTotal(), is(totalBlockBefore));
restartCluster(StartupOption.REGULAR);
assertThat("Total block count should be unchanged from start-up",
cluster.getNamesystem().getBlocksTotal(), is(totalBlockBefore));
checkFullFile(p, newLengthBeforeUpgrade, contents);
assertFileLength(snapshotFile, startingFileSize);
// empty edits and restart
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
fs.saveNamespace();
cluster.restartNameNode(true);
assertThat("Total block count should be unchanged from start-up",
cluster.getNamesystem().getBlocksTotal(), is(totalBlockBefore));
checkFullFile(p, newLengthBeforeUpgrade, contents);
assertFileLength(snapshotFile, startingFileSize);
fs.deleteSnapshot(parent, "ss0");
fs.delete(parent, true);
assertThat("File " + p + " shouldn't exist", fs.exists(p), is(false));
}
/** /**
* Check truncate recovery. * Check truncate recovery.
*/ */
@Test @Test
public void testTruncateLastBlock() throws IOException { public void testTruncateRecovery() throws IOException {
FSNamesystem fsn = cluster.getNamesystem(); FSNamesystem fsn = cluster.getNamesystem();
String client = "client";
String src = "/file"; String clientMachine = "clientMachine";
Path parent = new Path("/test");
String src = "/test/testTruncateRecovery";
Path srcPath = new Path(src); Path srcPath = new Path(src);
byte[] contents = AppendTestUtil.initBuffer(BLOCK_SIZE); byte[] contents = AppendTestUtil.initBuffer(BLOCK_SIZE);
writeContents(contents, BLOCK_SIZE, srcPath); writeContents(contents, BLOCK_SIZE, srcPath);
INodeFile inode = fsn.getFSDirectory().getINode(src).asFile(); INodesInPath iip = fsn.getFSDirectory().getINodesInPath4Write(src, true);
long oldGenstamp = GenerationStamp.LAST_RESERVED_STAMP; INodeFile file = iip.getLastINode().asFile();
DatanodeDescriptor dn = DFSTestUtil.getLocalDatanodeDescriptor(); long initialGenStamp = file.getLastBlock().getGenerationStamp();
DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo( // Test that prepareFileForTruncate sets up in-place truncate.
dn.getDatanodeUuid(), InetAddress.getLocalHost().getHostAddress());
dn.isAlive = true;
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
new Block(0, 1, oldGenstamp), (short) 1,
HdfsServerConstants.BlockUCState.BEING_TRUNCATED,
new DatanodeStorageInfo[] {storage});
inode.setBlocks(new BlockInfo[] {blockInfo});
fsn.writeLock(); fsn.writeLock();
try { try {
fsn.initializeBlockRecovery(inode); Block oldBlock = file.getLastBlock();
assertThat(inode.getLastBlock().getBlockUCState(), Block truncateBlock =
is(HdfsServerConstants.BlockUCState.BEING_TRUNCATED)); fsn.prepareFileForTruncate(iip, client, clientMachine, 1, null);
long blockRecoveryId = ((BlockInfoUnderConstruction) inode.getLastBlock()) // In-place truncate uses old block id with new genStamp.
assertThat(truncateBlock.getBlockId(),
is(equalTo(oldBlock.getBlockId())));
assertThat(truncateBlock.getNumBytes(),
is(oldBlock.getNumBytes()));
assertThat(truncateBlock.getGenerationStamp(),
is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
.getBlockRecoveryId(); .getBlockRecoveryId();
assertThat(blockRecoveryId, is(oldGenstamp + 2)); assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate(
src, client, clientMachine, BLOCK_SIZE-1, Time.now(), truncateBlock);
} finally { } finally {
fsn.writeUnlock(); fsn.writeUnlock();
} }
// Re-create file and ensure we are ready to copy on truncate
writeContents(contents, BLOCK_SIZE, srcPath);
fs.allowSnapshot(parent);
fs.createSnapshot(parent, "ss0");
iip = fsn.getFSDirectory().getINodesInPath(src, true);
file = iip.getLastINode().asFile();
file.recordModification(iip.getLatestSnapshotId(), true);
assertThat(file.isBlockInLatestSnapshot(file.getLastBlock()), is(true));
initialGenStamp = file.getLastBlock().getGenerationStamp();
// Test that prepareFileForTruncate sets up copy-on-write truncate
fsn.writeLock();
try {
Block oldBlock = file.getLastBlock();
Block truncateBlock =
fsn.prepareFileForTruncate(iip, client, clientMachine, 1, null);
// Copy-on-write truncate makes new block with new id and genStamp
assertThat(truncateBlock.getBlockId(),
is(not(equalTo(oldBlock.getBlockId()))));
assertThat(truncateBlock.getNumBytes() < oldBlock.getNumBytes(),
is(true));
assertThat(truncateBlock.getGenerationStamp(),
is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
.getBlockRecoveryId();
assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate(
src, client, clientMachine, BLOCK_SIZE-1, Time.now(), truncateBlock);
} finally {
fsn.writeUnlock();
}
checkBlockRecovery(srcPath);
fs.deleteSnapshot(parent, "ss0");
fs.delete(parent, true);
} }
static void writeContents(byte[] contents, int fileLength, Path p) static void writeContents(byte[] contents, int fileLength, Path p)
@ -286,4 +703,38 @@ public class TestFileTruncate {
static LocatedBlocks getLocatedBlocks(Path src) throws IOException { static LocatedBlocks getLocatedBlocks(Path src) throws IOException {
return fs.getClient().getLocatedBlocks(src.toString(), 0, Long.MAX_VALUE); return fs.getClient().getLocatedBlocks(src.toString(), 0, Long.MAX_VALUE);
} }
static void assertBlockExists(Block blk) {
assertNotNull("BlocksMap does not contain block: " + blk,
cluster.getNamesystem().getStoredBlock(blk));
}
static void assertBlockNotPresent(Block blk) {
assertNull("BlocksMap should not contain block: " + blk,
cluster.getNamesystem().getStoredBlock(blk));
}
static void assertFileLength(Path file, long length) throws IOException {
byte[] data = DFSTestUtil.readFileBuffer(fs, file);
assertEquals("Wrong data size in snapshot.", length, data.length);
}
static void checkFullFile(Path p, int newLength, byte[] contents)
throws IOException {
AppendTestUtil.checkFullFile(fs, p, newLength, contents, p.toString());
}
static void restartCluster(StartupOption o)
throws IOException {
cluster.shutdown();
if(StartupOption.ROLLBACK == o)
NameNode.doRollback(conf, false);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM)
.format(false)
.nameNodePort(NameNode.DEFAULT_PORT)
.startupOption(o==StartupOption.ROLLBACK ? StartupOption.REGULAR : o)
.dnStartupOption(o!=StartupOption.ROLLBACK ? StartupOption.REGULAR : o)
.build();
fs = cluster.getFileSystem();
}
} }

View File

@ -1,6 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?> <?xml version="1.0" encoding="UTF-8"?>
<EDITS> <EDITS>
<EDITS_VERSION>-60</EDITS_VERSION> <EDITS_VERSION>-61</EDITS_VERSION>
<RECORD> <RECORD>
<OPCODE>OP_START_LOG_SEGMENT</OPCODE> <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
<DATA> <DATA>
@ -13,8 +13,8 @@
<TXID>2</TXID> <TXID>2</TXID>
<DELEGATION_KEY> <DELEGATION_KEY>
<KEY_ID>1</KEY_ID> <KEY_ID>1</KEY_ID>
<EXPIRY_DATE>1421822547136</EXPIRY_DATE> <EXPIRY_DATE>1421826999207</EXPIRY_DATE>
<KEY>24319c7d1f7c0828</KEY> <KEY>ca9a0c8b240570b3</KEY>
</DELEGATION_KEY> </DELEGATION_KEY>
</DATA> </DATA>
</RECORD> </RECORD>
@ -24,8 +24,8 @@
<TXID>3</TXID> <TXID>3</TXID>
<DELEGATION_KEY> <DELEGATION_KEY>
<KEY_ID>2</KEY_ID> <KEY_ID>2</KEY_ID>
<EXPIRY_DATE>1421822547140</EXPIRY_DATE> <EXPIRY_DATE>1421826999210</EXPIRY_DATE>
<KEY>254b1207021431f4</KEY> <KEY>833c25a6fb2b0a6f</KEY>
</DELEGATION_KEY> </DELEGATION_KEY>
</DATA> </DATA>
</RECORD> </RECORD>
@ -37,19 +37,19 @@
<INODEID>16386</INODEID> <INODEID>16386</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348286</MTIME> <MTIME>1421135800328</MTIME>
<ATIME>1421131348286</ATIME> <ATIME>1421135800328</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>6</RPC_CALLID> <RPC_CALLID>9</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -60,14 +60,14 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348328</MTIME> <MTIME>1421135800357</MTIME>
<ATIME>1421131348286</ATIME> <ATIME>1421135800328</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
<OVERWRITE>false</OVERWRITE> <OVERWRITE>false</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -88,9 +88,9 @@
<LENGTH>0</LENGTH> <LENGTH>0</LENGTH>
<SRC>/file_create</SRC> <SRC>/file_create</SRC>
<DST>/file_moved</DST> <DST>/file_moved</DST>
<TIMESTAMP>1421131348343</TIMESTAMP> <TIMESTAMP>1421135800368</TIMESTAMP>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>9</RPC_CALLID> <RPC_CALLID>12</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -99,9 +99,9 @@
<TXID>8</TXID> <TXID>8</TXID>
<LENGTH>0</LENGTH> <LENGTH>0</LENGTH>
<PATH>/file_moved</PATH> <PATH>/file_moved</PATH>
<TIMESTAMP>1421131348353</TIMESTAMP> <TIMESTAMP>1421135800377</TIMESTAMP>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>10</RPC_CALLID> <RPC_CALLID>13</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -111,9 +111,9 @@
<LENGTH>0</LENGTH> <LENGTH>0</LENGTH>
<INODEID>16387</INODEID> <INODEID>16387</INODEID>
<PATH>/directory_mkdir</PATH> <PATH>/directory_mkdir</PATH>
<TIMESTAMP>1421131348366</TIMESTAMP> <TIMESTAMP>1421135800394</TIMESTAMP>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>493</MODE> <MODE>493</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -146,8 +146,8 @@
<TXID>13</TXID> <TXID>13</TXID>
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT> <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
<SNAPSHOTNAME>snapshot1</SNAPSHOTNAME> <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>15</RPC_CALLID> <RPC_CALLID>18</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -157,8 +157,8 @@
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT> <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
<SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME> <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
<SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME> <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>16</RPC_CALLID> <RPC_CALLID>19</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -167,8 +167,8 @@
<TXID>15</TXID> <TXID>15</TXID>
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT> <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
<SNAPSHOTNAME>snapshot2</SNAPSHOTNAME> <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>17</RPC_CALLID> <RPC_CALLID>20</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -179,19 +179,19 @@
<INODEID>16388</INODEID> <INODEID>16388</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348401</MTIME> <MTIME>1421135800442</MTIME>
<ATIME>1421131348401</ATIME> <ATIME>1421135800442</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>18</RPC_CALLID> <RPC_CALLID>21</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -202,14 +202,14 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348405</MTIME> <MTIME>1421135800445</MTIME>
<ATIME>1421131348401</ATIME> <ATIME>1421135800442</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
<OVERWRITE>false</OVERWRITE> <OVERWRITE>false</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -265,10 +265,10 @@
<LENGTH>0</LENGTH> <LENGTH>0</LENGTH>
<SRC>/file_create</SRC> <SRC>/file_create</SRC>
<DST>/file_moved</DST> <DST>/file_moved</DST>
<TIMESTAMP>1421131348436</TIMESTAMP> <TIMESTAMP>1421135800485</TIMESTAMP>
<OPTIONS>NONE</OPTIONS> <OPTIONS>NONE</OPTIONS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>25</RPC_CALLID> <RPC_CALLID>28</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -279,19 +279,19 @@
<INODEID>16389</INODEID> <INODEID>16389</INODEID>
<PATH>/file_concat_target</PATH> <PATH>/file_concat_target</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348443</MTIME> <MTIME>1421135800495</MTIME>
<ATIME>1421131348443</ATIME> <ATIME>1421135800495</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>27</RPC_CALLID> <RPC_CALLID>30</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -396,8 +396,8 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_concat_target</PATH> <PATH>/file_concat_target</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131348998</MTIME> <MTIME>1421135801050</MTIME>
<ATIME>1421131348443</ATIME> <ATIME>1421135800495</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
@ -418,7 +418,7 @@
<GENSTAMP>1003</GENSTAMP> <GENSTAMP>1003</GENSTAMP>
</BLOCK> </BLOCK>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -432,19 +432,19 @@
<INODEID>16390</INODEID> <INODEID>16390</INODEID>
<PATH>/file_concat_0</PATH> <PATH>/file_concat_0</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349001</MTIME> <MTIME>1421135801053</MTIME>
<ATIME>1421131349001</ATIME> <ATIME>1421135801053</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>38</RPC_CALLID> <RPC_CALLID>41</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -549,8 +549,8 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_concat_0</PATH> <PATH>/file_concat_0</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349032</MTIME> <MTIME>1421135801091</MTIME>
<ATIME>1421131349001</ATIME> <ATIME>1421135801053</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
@ -571,7 +571,7 @@
<GENSTAMP>1006</GENSTAMP> <GENSTAMP>1006</GENSTAMP>
</BLOCK> </BLOCK>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -585,19 +585,19 @@
<INODEID>16391</INODEID> <INODEID>16391</INODEID>
<PATH>/file_concat_1</PATH> <PATH>/file_concat_1</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349036</MTIME> <MTIME>1421135801095</MTIME>
<ATIME>1421131349036</ATIME> <ATIME>1421135801095</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>47</RPC_CALLID> <RPC_CALLID>50</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -702,8 +702,8 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_concat_1</PATH> <PATH>/file_concat_1</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349060</MTIME> <MTIME>1421135801126</MTIME>
<ATIME>1421131349036</ATIME> <ATIME>1421135801095</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
@ -724,7 +724,7 @@
<GENSTAMP>1009</GENSTAMP> <GENSTAMP>1009</GENSTAMP>
</BLOCK> </BLOCK>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -736,13 +736,13 @@
<TXID>57</TXID> <TXID>57</TXID>
<LENGTH>0</LENGTH> <LENGTH>0</LENGTH>
<TRG>/file_concat_target</TRG> <TRG>/file_concat_target</TRG>
<TIMESTAMP>1421131349064</TIMESTAMP> <TIMESTAMP>1421135801130</TIMESTAMP>
<SOURCES> <SOURCES>
<SOURCE1>/file_concat_0</SOURCE1> <SOURCE1>/file_concat_0</SOURCE1>
<SOURCE2>/file_concat_1</SOURCE2> <SOURCE2>/file_concat_1</SOURCE2>
</SOURCES> </SOURCES>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>55</RPC_CALLID> <RPC_CALLID>58</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -753,19 +753,19 @@
<INODEID>16392</INODEID> <INODEID>16392</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349068</MTIME> <MTIME>1421135810102</MTIME>
<ATIME>1421131349068</ATIME> <ATIME>1421135810102</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>57</RPC_CALLID> <RPC_CALLID>63</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -837,8 +837,8 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/file_create</PATH> <PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349085</MTIME> <MTIME>1421135810122</MTIME>
<ATIME>1421131349068</ATIME> <ATIME>1421135810102</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
@ -854,7 +854,7 @@
<GENSTAMP>1011</GENSTAMP> <GENSTAMP>1011</GENSTAMP>
</BLOCK> </BLOCK>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -865,10 +865,10 @@
<DATA> <DATA>
<TXID>66</TXID> <TXID>66</TXID>
<SRC>/file_create</SRC> <SRC>/file_create</SRC>
<CLIENTNAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENTNAME> <CLIENTNAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENTNAME>
<CLIENTMACHINE>127.0.0.1</CLIENTMACHINE> <CLIENTMACHINE>127.0.0.1</CLIENTMACHINE>
<NEWLENGTH>512</NEWLENGTH> <NEWLENGTH>512</NEWLENGTH>
<TIMESTAMP>1421131349088</TIMESTAMP> <TIMESTAMP>1421135810125</TIMESTAMP>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -879,15 +879,15 @@
<INODEID>16393</INODEID> <INODEID>16393</INODEID>
<PATH>/file_symlink</PATH> <PATH>/file_symlink</PATH>
<VALUE>/file_concat_target</VALUE> <VALUE>/file_concat_target</VALUE>
<MTIME>1421131349095</MTIME> <MTIME>1421135810132</MTIME>
<ATIME>1421131349095</ATIME> <ATIME>1421135810132</ATIME>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>511</MODE> <MODE>511</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>64</RPC_CALLID> <RPC_CALLID>70</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -898,19 +898,19 @@
<INODEID>16394</INODEID> <INODEID>16394</INODEID>
<PATH>/hard-lease-recovery-test</PATH> <PATH>/hard-lease-recovery-test</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131349098</MTIME> <MTIME>1421135810135</MTIME>
<ATIME>1421131349098</ATIME> <ATIME>1421135810135</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME>DFSClient_NONMAPREDUCE_526346936_1</CLIENT_NAME> <CLIENT_NAME>DFSClient_NONMAPREDUCE_240777107_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE> <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
<OVERWRITE>true</OVERWRITE> <OVERWRITE>true</OVERWRITE>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>65</RPC_CALLID> <RPC_CALLID>71</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -966,7 +966,7 @@
<OPCODE>OP_REASSIGN_LEASE</OPCODE> <OPCODE>OP_REASSIGN_LEASE</OPCODE>
<DATA> <DATA>
<TXID>74</TXID> <TXID>74</TXID>
<LEASEHOLDER>DFSClient_NONMAPREDUCE_526346936_1</LEASEHOLDER> <LEASEHOLDER>DFSClient_NONMAPREDUCE_240777107_1</LEASEHOLDER>
<PATH>/hard-lease-recovery-test</PATH> <PATH>/hard-lease-recovery-test</PATH>
<NEWHOLDER>HDFS_NameNode</NEWHOLDER> <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
</DATA> </DATA>
@ -979,8 +979,8 @@
<INODEID>0</INODEID> <INODEID>0</INODEID>
<PATH>/hard-lease-recovery-test</PATH> <PATH>/hard-lease-recovery-test</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<MTIME>1421131351230</MTIME> <MTIME>1421135812235</MTIME>
<ATIME>1421131349098</ATIME> <ATIME>1421135810135</ATIME>
<BLOCKSIZE>512</BLOCKSIZE> <BLOCKSIZE>512</BLOCKSIZE>
<CLIENT_NAME></CLIENT_NAME> <CLIENT_NAME></CLIENT_NAME>
<CLIENT_MACHINE></CLIENT_MACHINE> <CLIENT_MACHINE></CLIENT_MACHINE>
@ -991,7 +991,7 @@
<GENSTAMP>1013</GENSTAMP> <GENSTAMP>1013</GENSTAMP>
</BLOCK> </BLOCK>
<PERMISSION_STATUS> <PERMISSION_STATUS>
<USERNAME>plamenjeliazkov</USERNAME> <USERNAME>shv</USERNAME>
<GROUPNAME>supergroup</GROUPNAME> <GROUPNAME>supergroup</GROUPNAME>
<MODE>420</MODE> <MODE>420</MODE>
</PERMISSION_STATUS> </PERMISSION_STATUS>
@ -1002,13 +1002,13 @@
<DATA> <DATA>
<TXID>76</TXID> <TXID>76</TXID>
<POOLNAME>pool1</POOLNAME> <POOLNAME>pool1</POOLNAME>
<OWNERNAME>plamenjeliazkov</OWNERNAME> <OWNERNAME>shv</OWNERNAME>
<GROUPNAME>staff</GROUPNAME> <GROUPNAME>shv</GROUPNAME>
<MODE>493</MODE> <MODE>493</MODE>
<LIMIT>9223372036854775807</LIMIT> <LIMIT>9223372036854775807</LIMIT>
<MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY> <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>72</RPC_CALLID> <RPC_CALLID>78</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1017,8 +1017,8 @@
<TXID>77</TXID> <TXID>77</TXID>
<POOLNAME>pool1</POOLNAME> <POOLNAME>pool1</POOLNAME>
<LIMIT>99</LIMIT> <LIMIT>99</LIMIT>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>73</RPC_CALLID> <RPC_CALLID>79</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1029,9 +1029,9 @@
<PATH>/path</PATH> <PATH>/path</PATH>
<REPLICATION>1</REPLICATION> <REPLICATION>1</REPLICATION>
<POOL>pool1</POOL> <POOL>pool1</POOL>
<EXPIRATION>2305844430345046085</EXPIRATION> <EXPIRATION>2305844430349507141</EXPIRATION>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>74</RPC_CALLID> <RPC_CALLID>80</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1040,8 +1040,8 @@
<TXID>79</TXID> <TXID>79</TXID>
<ID>1</ID> <ID>1</ID>
<REPLICATION>2</REPLICATION> <REPLICATION>2</REPLICATION>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>75</RPC_CALLID> <RPC_CALLID>81</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1049,8 +1049,8 @@
<DATA> <DATA>
<TXID>80</TXID> <TXID>80</TXID>
<ID>1</ID> <ID>1</ID>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>76</RPC_CALLID> <RPC_CALLID>82</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1058,8 +1058,8 @@
<DATA> <DATA>
<TXID>81</TXID> <TXID>81</TXID>
<POOLNAME>pool1</POOLNAME> <POOLNAME>pool1</POOLNAME>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>77</RPC_CALLID> <RPC_CALLID>83</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1105,8 +1105,8 @@
<NAME>a1</NAME> <NAME>a1</NAME>
<VALUE>0x313233</VALUE> <VALUE>0x313233</VALUE>
</XATTR> </XATTR>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>79</RPC_CALLID> <RPC_CALLID>85</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1119,8 +1119,8 @@
<NAME>a2</NAME> <NAME>a2</NAME>
<VALUE>0x373839</VALUE> <VALUE>0x373839</VALUE>
</XATTR> </XATTR>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>80</RPC_CALLID> <RPC_CALLID>86</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
@ -1132,22 +1132,22 @@
<NAMESPACE>USER</NAMESPACE> <NAMESPACE>USER</NAMESPACE>
<NAME>a2</NAME> <NAME>a2</NAME>
</XATTR> </XATTR>
<RPC_CLIENTID>99bcddc1-3460-4630-9904-6c7ca5811945</RPC_CLIENTID> <RPC_CLIENTID>cb20a92a-2c2f-4305-a838-2a01c6e73e18</RPC_CLIENTID>
<RPC_CALLID>81</RPC_CALLID> <RPC_CALLID>87</RPC_CALLID>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
<OPCODE>OP_ROLLING_UPGRADE_START</OPCODE> <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
<DATA> <DATA>
<TXID>86</TXID> <TXID>86</TXID>
<STARTTIME>1421131352186</STARTTIME> <STARTTIME>1421135813268</STARTTIME>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>
<OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE> <OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE>
<DATA> <DATA>
<TXID>87</TXID> <TXID>87</TXID>
<FINALIZETIME>1421131352186</FINALIZETIME> <FINALIZETIME>1421135813268</FINALIZETIME>
</DATA> </DATA>
</RECORD> </RECORD>
<RECORD> <RECORD>