HDFS-7056. Snapshot support for truncate. Contributed by Konstantin Shvachko and Plamen Jeliazkov.
This commit is contained in:
parent
6ff9bde714
commit
00a7ebab22
|
@ -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)
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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(
|
||||||
|
|
|
@ -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*"
|
||||||
|
|
|
@ -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";
|
||||||
|
|
|
@ -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 };
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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="
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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: {
|
||||||
|
|
|
@ -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=");
|
||||||
|
|
|
@ -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");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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}
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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()))
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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++]);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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];
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Binary file not shown.
|
@ -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>
|
||||||
|
|
Loading…
Reference in New Issue