|
@@ -40,9 +40,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
|
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
|
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
@@ -84,12 +82,14 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
|
|
|
|
/**
|
|
/**
|
|
* Bit format:
|
|
* Bit format:
|
|
- * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
|
|
|
|
|
|
+ * [4-bit storagePolicyID][1-bit isStriped]
|
|
|
|
+ * [11-bit replication][48-bit preferredBlockSize]
|
|
*/
|
|
*/
|
|
static enum HeaderFormat {
|
|
static enum HeaderFormat {
|
|
PREFERRED_BLOCK_SIZE(null, 48, 1),
|
|
PREFERRED_BLOCK_SIZE(null, 48, 1),
|
|
- REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
|
|
|
|
- STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
|
|
|
|
|
|
+ REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
|
|
|
|
+ IS_STRIPED(REPLICATION.BITS, 1, 0),
|
|
|
|
+ STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
|
|
0);
|
|
0);
|
|
|
|
|
|
private final LongBitFormat BITS;
|
|
private final LongBitFormat BITS;
|
|
@@ -110,14 +110,27 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
|
|
return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ static boolean isStriped(long header) {
|
|
|
|
+ long isStriped = IS_STRIPED.BITS.retrieve(header);
|
|
|
|
+ Preconditions.checkState(isStriped == 0 || isStriped == 1);
|
|
|
|
+ return isStriped == 1;
|
|
|
|
+ }
|
|
|
|
+
|
|
static long toLong(long preferredBlockSize, short replication,
|
|
static long toLong(long preferredBlockSize, short replication,
|
|
- byte storagePolicyID) {
|
|
|
|
|
|
+ boolean isStriped, byte storagePolicyID) {
|
|
long h = 0;
|
|
long h = 0;
|
|
if (preferredBlockSize == 0) {
|
|
if (preferredBlockSize == 0) {
|
|
preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
|
|
preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
|
|
}
|
|
}
|
|
h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
|
|
h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
|
|
- h = REPLICATION.BITS.combine(replication, h);
|
|
|
|
|
|
+ // Replication factor for striped files is zero
|
|
|
|
+ if (isStriped) {
|
|
|
|
+ h = REPLICATION.BITS.combine(0L, h);
|
|
|
|
+ h = IS_STRIPED.BITS.combine(1L, h);
|
|
|
|
+ } else {
|
|
|
|
+ h = REPLICATION.BITS.combine(replication, h);
|
|
|
|
+ h = IS_STRIPED.BITS.combine(0L, h);
|
|
|
|
+ }
|
|
h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
|
|
h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
|
|
return h;
|
|
return h;
|
|
}
|
|
}
|
|
@@ -126,21 +139,27 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
|
|
|
|
private long header = 0L;
|
|
private long header = 0L;
|
|
|
|
|
|
- private BlockInfoContiguous[] blocks;
|
|
|
|
|
|
+ private BlockInfo[] blocks;
|
|
|
|
|
|
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
|
|
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
|
|
- long atime, BlockInfoContiguous[] blklist, short replication,
|
|
|
|
|
|
+ long atime, BlockInfo[] blklist, short replication,
|
|
long preferredBlockSize) {
|
|
long preferredBlockSize) {
|
|
this(id, name, permissions, mtime, atime, blklist, replication,
|
|
this(id, name, permissions, mtime, atime, blklist, replication,
|
|
- preferredBlockSize, (byte) 0);
|
|
|
|
|
|
+ preferredBlockSize, (byte) 0, false);
|
|
}
|
|
}
|
|
|
|
|
|
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
|
|
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
|
|
- long atime, BlockInfoContiguous[] blklist, short replication,
|
|
|
|
- long preferredBlockSize, byte storagePolicyID) {
|
|
|
|
|
|
+ long atime, BlockInfo[] blklist, short replication,
|
|
|
|
+ long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
|
|
super(id, name, permissions, mtime, atime);
|
|
super(id, name, permissions, mtime, atime);
|
|
- header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
|
|
|
|
- this.blocks = blklist;
|
|
|
|
|
|
+ header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
|
|
|
|
+ storagePolicyID);
|
|
|
|
+ if (blklist != null && blklist.length > 0) {
|
|
|
|
+ for (BlockInfo b : blklist) {
|
|
|
|
+ Preconditions.checkArgument(b.isStriped() == isStriped);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ setBlocks(blklist);
|
|
}
|
|
}
|
|
|
|
|
|
public INodeFile(INodeFile that) {
|
|
public INodeFile(INodeFile that) {
|
|
@@ -177,27 +196,6 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
&& getXAttrFeature() == other.getXAttrFeature();
|
|
&& getXAttrFeature() == other.getXAttrFeature();
|
|
}
|
|
}
|
|
|
|
|
|
- /* Start of StripedBlock Feature */
|
|
|
|
-
|
|
|
|
- public final FileWithStripedBlocksFeature getStripedBlocksFeature() {
|
|
|
|
- return getFeature(FileWithStripedBlocksFeature.class);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public FileWithStripedBlocksFeature addStripedBlocksFeature() {
|
|
|
|
- assert blocks == null || blocks.length == 0:
|
|
|
|
- "The file contains contiguous blocks";
|
|
|
|
- assert !isStriped();
|
|
|
|
- this.setFileReplication((short) 0);
|
|
|
|
- FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
|
|
|
|
- addFeature(sb);
|
|
|
|
- return sb;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Used to make sure there is no contiguous block related info */
|
|
|
|
- private boolean hasNoContiguousBlock() {
|
|
|
|
- return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/* Start of Under-Construction Feature */
|
|
/* Start of Under-Construction Feature */
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -232,7 +230,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
"file is no longer under construction");
|
|
"file is no longer under construction");
|
|
FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
|
|
FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
|
|
if (uc != null) {
|
|
if (uc != null) {
|
|
- assertAllBlocksComplete(getBlocks());
|
|
|
|
|
|
+ assertAllBlocksComplete();
|
|
removeFeature(uc);
|
|
removeFeature(uc);
|
|
this.setModificationTime(mtime);
|
|
this.setModificationTime(mtime);
|
|
}
|
|
}
|
|
@@ -240,32 +238,21 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
/** Assert all blocks are complete. */
|
|
/** Assert all blocks are complete. */
|
|
- private void assertAllBlocksComplete(BlockInfo[] blks) {
|
|
|
|
- if (blks == null) {
|
|
|
|
|
|
+ private void assertAllBlocksComplete() {
|
|
|
|
+ if (blocks == null) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- for (int i = 0; i < blks.length; i++) {
|
|
|
|
- Preconditions.checkState(blks[i].isComplete(), "Failed to finalize"
|
|
|
|
|
|
+ for (int i = 0; i < blocks.length; i++) {
|
|
|
|
+ Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
|
|
+ " %s %s since blocks[%s] is non-complete, where blocks=%s.",
|
|
+ " %s %s since blocks[%s] is non-complete, where blocks=%s.",
|
|
- getClass().getSimpleName(), this, i, Arrays.asList(blks));
|
|
|
|
|
|
+ getClass().getSimpleName(), this, i, Arrays.asList(blocks));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Instead of adding a new block, this function is usually used while loading
|
|
|
|
- * fsimage or converting the last block to UC/Complete.
|
|
|
|
- */
|
|
|
|
@Override // BlockCollection
|
|
@Override // BlockCollection
|
|
public void setBlock(int index, BlockInfo blk) {
|
|
public void setBlock(int index, BlockInfo blk) {
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- assert !blk.isStriped();
|
|
|
|
- this.blocks[index] = (BlockInfoContiguous) blk;
|
|
|
|
- } else {
|
|
|
|
- assert blk.isStriped();
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- sb.setBlock(index, (BlockInfoStriped) blk);
|
|
|
|
- }
|
|
|
|
|
|
+ Preconditions.checkArgument(blk.isStriped() == this.isStriped());
|
|
|
|
+ this.blocks[index] = blk;
|
|
}
|
|
}
|
|
|
|
|
|
@Override // BlockCollection, the file should be under construction
|
|
@Override // BlockCollection, the file should be under construction
|
|
@@ -278,15 +265,13 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
final BlockInfo ucBlock;
|
|
final BlockInfo ucBlock;
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- assert !lastBlock.isStriped();
|
|
|
|
- ucBlock = ((BlockInfoContiguous) lastBlock)
|
|
|
|
|
|
+ if (isStriped()) {
|
|
|
|
+ Preconditions.checkState(lastBlock.isStriped());
|
|
|
|
+ ucBlock = ((BlockInfoStriped) lastBlock)
|
|
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
|
|
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
|
|
} else {
|
|
} else {
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- assert lastBlock.isStriped();
|
|
|
|
- ucBlock = ((BlockInfoStriped) lastBlock)
|
|
|
|
|
|
+ Preconditions.checkState(!lastBlock.isStriped());
|
|
|
|
+ ucBlock = ((BlockInfoContiguous) lastBlock)
|
|
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
|
|
.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
|
|
}
|
|
}
|
|
setBlock(numBlocks() - 1, ucBlock);
|
|
setBlock(numBlocks() - 1, ucBlock);
|
|
@@ -299,27 +284,21 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
|
|
BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
|
|
Preconditions.checkState(isUnderConstruction(),
|
|
Preconditions.checkState(isUnderConstruction(),
|
|
"file is no longer under construction");
|
|
"file is no longer under construction");
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- if (blocks == null || blocks.length == 0) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- int size_1 = blocks.length - 1;
|
|
|
|
- if (!blocks[size_1].equals(oldblock)) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- BlockInfoContiguousUnderConstruction uc =
|
|
|
|
- (BlockInfoContiguousUnderConstruction)blocks[size_1];
|
|
|
|
- //copy to a new list
|
|
|
|
- BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
|
|
|
|
- System.arraycopy(blocks, 0, newlist, 0, size_1);
|
|
|
|
- setContiguousBlocks(newlist);
|
|
|
|
- return uc;
|
|
|
|
- } else {
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- return sb.removeLastBlock(oldblock);
|
|
|
|
|
|
+ if (blocks == null || blocks.length == 0) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ int size_1 = blocks.length - 1;
|
|
|
|
+ if (!blocks[size_1].equals(oldblock)) {
|
|
|
|
+ return null;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ BlockInfoUnderConstruction uc =
|
|
|
|
+ (BlockInfoUnderConstruction)blocks[size_1];
|
|
|
|
+ //copy to a new list
|
|
|
|
+ BlockInfo[] newlist = new BlockInfo[size_1];
|
|
|
|
+ System.arraycopy(blocks, 0, newlist, 0, size_1);
|
|
|
|
+ setBlocks(newlist);
|
|
|
|
+ return uc;
|
|
}
|
|
}
|
|
|
|
|
|
/* End of Under-Construction Feature */
|
|
/* End of Under-Construction Feature */
|
|
@@ -401,7 +380,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
|
|
|
|
/** The same as getFileReplication(null). */
|
|
/** The same as getFileReplication(null). */
|
|
@Override // INodeFileAttributes
|
|
@Override // INodeFileAttributes
|
|
- // TODO striped
|
|
|
|
|
|
+ // TODO properly handle striped files
|
|
public final short getFileReplication() {
|
|
public final short getFileReplication() {
|
|
return getFileReplication(CURRENT_STATE_ID);
|
|
return getFileReplication(CURRENT_STATE_ID);
|
|
}
|
|
}
|
|
@@ -466,6 +445,16 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
setStoragePolicyID(storagePolicyId);
|
|
setStoragePolicyID(storagePolicyId);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * @return true if the file is in the striping layout.
|
|
|
|
+ */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ @Override
|
|
|
|
+ public boolean isStriped() {
|
|
|
|
+ return HeaderFormat.isStriped(header);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override // INodeFileAttributes
|
|
@Override // INodeFileAttributes
|
|
public long getHeaderLong() {
|
|
public long getHeaderLong() {
|
|
return header;
|
|
return header;
|
|
@@ -474,17 +463,6 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
/** @return the blocks of the file. */
|
|
/** @return the blocks of the file. */
|
|
@Override // BlockCollection
|
|
@Override // BlockCollection
|
|
public BlockInfo[] getBlocks() {
|
|
public BlockInfo[] getBlocks() {
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb != null) {
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- return sb.getBlocks();
|
|
|
|
- } else {
|
|
|
|
- return this.blocks;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Used by snapshot diff */
|
|
|
|
- public BlockInfoContiguous[] getContiguousBlocks() {
|
|
|
|
return this.blocks;
|
|
return this.blocks;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -507,16 +485,11 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
/** Used during concat to update the BlockCollection for each block */
|
|
/** Used during concat to update the BlockCollection for each block */
|
|
- private void updateBlockCollection() {
|
|
|
|
- if (blocks != null && blocks.length > 0) {
|
|
|
|
- for(BlockInfoContiguous b : blocks) {
|
|
|
|
|
|
+ void updateBlockCollection() {
|
|
|
|
+ if (blocks != null) {
|
|
|
|
+ for(BlockInfo b : blocks) {
|
|
b.setBlockCollection(this);
|
|
b.setBlockCollection(this);
|
|
}
|
|
}
|
|
- } else {
|
|
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb != null) {
|
|
|
|
- sb.updateBlockCollection(this);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -527,11 +500,12 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
int size = this.blocks.length;
|
|
int size = this.blocks.length;
|
|
int totalAddedBlocks = 0;
|
|
int totalAddedBlocks = 0;
|
|
for(INodeFile f : inodes) {
|
|
for(INodeFile f : inodes) {
|
|
|
|
+ Preconditions.checkState(f.isStriped() == this.isStriped());
|
|
totalAddedBlocks += f.blocks.length;
|
|
totalAddedBlocks += f.blocks.length;
|
|
}
|
|
}
|
|
|
|
|
|
- BlockInfoContiguous[] newlist =
|
|
|
|
- new BlockInfoContiguous[size + totalAddedBlocks];
|
|
|
|
|
|
+ BlockInfo[] newlist =
|
|
|
|
+ new BlockInfo[size + totalAddedBlocks];
|
|
System.arraycopy(this.blocks, 0, newlist, 0, size);
|
|
System.arraycopy(this.blocks, 0, newlist, 0, size);
|
|
|
|
|
|
for(INodeFile in: inodes) {
|
|
for(INodeFile in: inodes) {
|
|
@@ -539,43 +513,35 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
size += in.blocks.length;
|
|
size += in.blocks.length;
|
|
}
|
|
}
|
|
|
|
|
|
- setContiguousBlocks(newlist);
|
|
|
|
|
|
+ setBlocks(newlist);
|
|
updateBlockCollection();
|
|
updateBlockCollection();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * add a contiguous block to the block list
|
|
|
|
|
|
+ * add a block to the block list
|
|
*/
|
|
*/
|
|
- private void addContiguousBlock(BlockInfoContiguous newblock) {
|
|
|
|
|
|
+ void addBlock(BlockInfo newblock) {
|
|
|
|
+ Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
|
|
if (this.blocks == null) {
|
|
if (this.blocks == null) {
|
|
- this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
|
|
|
|
|
|
+ this.setBlocks(new BlockInfo[]{newblock});
|
|
} else {
|
|
} else {
|
|
int size = this.blocks.length;
|
|
int size = this.blocks.length;
|
|
- BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
|
|
|
|
|
|
+ BlockInfo[] newlist = new BlockInfo[size + 1];
|
|
System.arraycopy(this.blocks, 0, newlist, 0, size);
|
|
System.arraycopy(this.blocks, 0, newlist, 0, size);
|
|
newlist[size] = newblock;
|
|
newlist[size] = newblock;
|
|
- this.setContiguousBlocks(newlist);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** add a striped or contiguous block */
|
|
|
|
- void addBlock(BlockInfo newblock) {
|
|
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- assert !newblock.isStriped();
|
|
|
|
- addContiguousBlock((BlockInfoContiguous) newblock);
|
|
|
|
- } else {
|
|
|
|
- assert newblock.isStriped();
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- sb.addBlock((BlockInfoStriped) newblock);
|
|
|
|
|
|
+ this.setBlocks(newlist);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/** Set the blocks. */
|
|
/** Set the blocks. */
|
|
- public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
|
|
|
|
|
|
+ private void setBlocks(BlockInfo[] blocks) {
|
|
this.blocks = blocks;
|
|
this.blocks = blocks;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void clearBlocks() {
|
|
|
|
+ setBlocks(null);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void cleanSubtree(ReclaimContext reclaimContext,
|
|
public void cleanSubtree(ReclaimContext reclaimContext,
|
|
final int snapshot, int priorSnapshotId) {
|
|
final int snapshot, int priorSnapshotId) {
|
|
@@ -623,19 +589,13 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
public void clearFile(ReclaimContext reclaimContext) {
|
|
public void clearFile(ReclaimContext reclaimContext) {
|
|
- BlockInfo[] blks = getBlocks();
|
|
|
|
- if (blks != null && reclaimContext.collectedBlocks != null) {
|
|
|
|
- for (BlockInfo blk : blks) {
|
|
|
|
|
|
+ if (blocks != null && reclaimContext.collectedBlocks != null) {
|
|
|
|
+ for (BlockInfo blk : blocks) {
|
|
reclaimContext.collectedBlocks.addDeleteBlock(blk);
|
|
reclaimContext.collectedBlocks.addDeleteBlock(blk);
|
|
blk.setBlockCollection(null);
|
|
blk.setBlockCollection(null);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- setContiguousBlocks(null);
|
|
|
|
-
|
|
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb != null) {
|
|
|
|
- sb.clear();
|
|
|
|
- }
|
|
|
|
|
|
+ clearBlocks();
|
|
if (getAclFeature() != null) {
|
|
if (getAclFeature() != null) {
|
|
AclStorage.removeAclFeature(getAclFeature());
|
|
AclStorage.removeAclFeature(getAclFeature());
|
|
}
|
|
}
|
|
@@ -677,7 +637,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
if (isStriped()) {
|
|
if (isStriped()) {
|
|
return computeQuotaUsageWithStriped(bsp, counts);
|
|
return computeQuotaUsageWithStriped(bsp, counts);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
if (last < lastSnapshotId) {
|
|
if (last < lastSnapshotId) {
|
|
ssDeltaNoReplication = computeFileSize(true, false);
|
|
ssDeltaNoReplication = computeFileSize(true, false);
|
|
replication = getFileReplication();
|
|
replication = getFileReplication();
|
|
@@ -790,33 +750,27 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
* @return file size
|
|
* @return file size
|
|
*/
|
|
*/
|
|
public final long computeFileSize(boolean includesLastUcBlock,
|
|
public final long computeFileSize(boolean includesLastUcBlock,
|
|
- boolean usePreferredBlockSize4LastUcBlock) {
|
|
|
|
- BlockInfo[] blockInfos = getBlocks();
|
|
|
|
- // In case of contiguous blocks
|
|
|
|
- if (blockInfos == null || blockInfos.length == 0) {
|
|
|
|
|
|
+ boolean usePreferredBlockSize4LastUcBlock) {
|
|
|
|
+ if (blocks == null || blocks.length == 0) {
|
|
return 0;
|
|
return 0;
|
|
}
|
|
}
|
|
- final int last = blockInfos.length - 1;
|
|
|
|
|
|
+ final int last = blocks.length - 1;
|
|
//check if the last block is BlockInfoUnderConstruction
|
|
//check if the last block is BlockInfoUnderConstruction
|
|
- long size = blockInfos[last].getNumBytes();
|
|
|
|
- if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
|
|
|
|
- if (!includesLastUcBlock) {
|
|
|
|
- size = 0;
|
|
|
|
- } else if (usePreferredBlockSize4LastUcBlock) {
|
|
|
|
- size = getPreferredBlockSize();
|
|
|
|
- }
|
|
|
|
- } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
|
|
|
|
- if (!includesLastUcBlock) {
|
|
|
|
- size = 0;
|
|
|
|
- } else if (usePreferredBlockSize4LastUcBlock) {
|
|
|
|
- BlockInfoStripedUnderConstruction blockInfoStripedUC
|
|
|
|
- = (BlockInfoStripedUnderConstruction) blockInfos[last];
|
|
|
|
- size = getPreferredBlockSize() * blockInfoStripedUC.getDataBlockNum();
|
|
|
|
- }
|
|
|
|
|
|
+ BlockInfo lastBlk = blocks[last];
|
|
|
|
+ long size = lastBlk.getNumBytes();
|
|
|
|
+ if (lastBlk instanceof BlockInfoUnderConstruction) {
|
|
|
|
+ if (!includesLastUcBlock) {
|
|
|
|
+ size = 0;
|
|
|
|
+ } else if (usePreferredBlockSize4LastUcBlock) {
|
|
|
|
+ size = isStriped()?
|
|
|
|
+ getPreferredBlockSize() *
|
|
|
|
+ ((BlockInfoStriped)lastBlk).getDataBlockNum() :
|
|
|
|
+ getPreferredBlockSize();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
//sum other blocks
|
|
//sum other blocks
|
|
for (int i = 0; i < last; i++) {
|
|
for (int i = 0; i < last; i++) {
|
|
- size += blockInfos[i].getNumBytes();
|
|
|
|
|
|
+ size += blocks[i].getNumBytes();
|
|
}
|
|
}
|
|
return size;
|
|
return size;
|
|
}
|
|
}
|
|
@@ -828,41 +782,30 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
*/
|
|
*/
|
|
public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
|
|
public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
|
|
if (isStriped()) {
|
|
if (isStriped()) {
|
|
- return storagespaceConsumedWithStriped();
|
|
|
|
|
|
+ return storagespaceConsumedStriped();
|
|
} else {
|
|
} else {
|
|
- return storagespaceConsumedWithReplication(bsp);
|
|
|
|
|
|
+ return storagespaceConsumedContiguous(bsp);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
// TODO: support EC with heterogeneous storage
|
|
// TODO: support EC with heterogeneous storage
|
|
- public final QuotaCounts storagespaceConsumedWithStriped() {
|
|
|
|
|
|
+ public final QuotaCounts storagespaceConsumedStriped() {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
- BlockInfo[] blockInfos = getBlocks();
|
|
|
|
- if (blockInfos == null || blockInfos.length == 0) {
|
|
|
|
|
|
+ if (blocks == null || blocks.length == 0) {
|
|
return counts;
|
|
return counts;
|
|
}
|
|
}
|
|
|
|
|
|
- long size;
|
|
|
|
- final int last = blockInfos.length - 1;
|
|
|
|
- if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
|
|
|
|
- BlockInfoStripedUnderConstruction blockInfoStripedUC
|
|
|
|
- =(BlockInfoStripedUnderConstruction)blockInfos[last];
|
|
|
|
- size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
|
|
|
|
- } else {
|
|
|
|
- // In case of last block is complete
|
|
|
|
- BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
|
|
|
|
- size = blockInfoStriped.spaceConsumed();
|
|
|
|
- }
|
|
|
|
- for (int i = 0; i < last; i++) {
|
|
|
|
- BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
|
|
|
|
- size += blockInfoStriped.spaceConsumed();
|
|
|
|
|
|
+ for (BlockInfo b : blocks) {
|
|
|
|
+ Preconditions.checkState(b.isStriped());
|
|
|
|
+ long blockSize = b.isComplete() ?
|
|
|
|
+ ((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() *
|
|
|
|
+ ((BlockInfoStriped)b).getTotalBlockNum();
|
|
|
|
+ counts.addStorageSpace(blockSize);
|
|
}
|
|
}
|
|
-
|
|
|
|
- counts.addStorageSpace(size);
|
|
|
|
return counts;
|
|
return counts;
|
|
}
|
|
}
|
|
|
|
|
|
- public final QuotaCounts storagespaceConsumedWithReplication(
|
|
|
|
|
|
+ public final QuotaCounts storagespaceConsumedContiguous(
|
|
BlockStoragePolicy bsp) {
|
|
BlockStoragePolicy bsp) {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
final Iterable<BlockInfo> blocks;
|
|
final Iterable<BlockInfo> blocks;
|
|
@@ -874,7 +817,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
|
|
Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
|
|
List<FileDiff> diffs = sf.getDiffs().asList();
|
|
List<FileDiff> diffs = sf.getDiffs().asList();
|
|
for(FileDiff diff : diffs) {
|
|
for(FileDiff diff : diffs) {
|
|
- BlockInfoContiguous[] diffBlocks = diff.getBlocks();
|
|
|
|
|
|
+ BlockInfo[] diffBlocks = diff.getBlocks();
|
|
if (diffBlocks != null) {
|
|
if (diffBlocks != null) {
|
|
allBlocks.addAll(Arrays.asList(diffBlocks));
|
|
allBlocks.addAll(Arrays.asList(diffBlocks));
|
|
}
|
|
}
|
|
@@ -903,32 +846,20 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
* Return the penultimate allocated block for this file.
|
|
* Return the penultimate allocated block for this file.
|
|
*/
|
|
*/
|
|
BlockInfo getPenultimateBlock() {
|
|
BlockInfo getPenultimateBlock() {
|
|
- BlockInfo[] blks = getBlocks();
|
|
|
|
- return (blks == null || blks.length <= 1) ?
|
|
|
|
- null : blks[blks.length - 2];
|
|
|
|
|
|
+ if (blocks == null || blocks.length <= 1) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ return blocks[blocks.length - 2];
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public BlockInfo getLastBlock() {
|
|
public BlockInfo getLastBlock() {
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- return blocks == null || blocks.length == 0 ?
|
|
|
|
- null : blocks[blocks.length - 1];
|
|
|
|
- } else {
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- return sb.getLastBlock();
|
|
|
|
- }
|
|
|
|
|
|
+ return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int numBlocks() {
|
|
public int numBlocks() {
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- return blocks == null ? 0 : blocks.length;
|
|
|
|
- } else {
|
|
|
|
- assert hasNoContiguousBlock();
|
|
|
|
- return sb.numBlocks();
|
|
|
|
- }
|
|
|
|
|
|
+ return blocks == null ? 0 : blocks.length;
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
@@ -939,8 +870,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
out.print(", fileSize=" + computeFileSize(snapshotId));
|
|
out.print(", fileSize=" + computeFileSize(snapshotId));
|
|
// only compare the first block
|
|
// only compare the first block
|
|
out.print(", blocks=");
|
|
out.print(", blocks=");
|
|
- BlockInfo[] blks = getBlocks();
|
|
|
|
- out.print(blks == null || blks.length == 0? null: blks[0]);
|
|
|
|
|
|
+ out.print(blocks == null || blocks.length == 0? null: blocks[0]);
|
|
out.println();
|
|
out.println();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1033,24 +963,15 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
void truncateBlocksTo(int n) {
|
|
void truncateBlocksTo(int n) {
|
|
- FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
|
|
- if (sb == null) {
|
|
|
|
- truncateContiguousBlocks(n);
|
|
|
|
- } else {
|
|
|
|
- sb.truncateStripedBlocks(n);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void truncateContiguousBlocks(int n) {
|
|
|
|
- final BlockInfoContiguous[] newBlocks;
|
|
|
|
|
|
+ final BlockInfo[] newBlocks;
|
|
if (n == 0) {
|
|
if (n == 0) {
|
|
- newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
|
|
|
|
|
|
+ newBlocks = BlockInfo.EMPTY_ARRAY;
|
|
} else {
|
|
} else {
|
|
- newBlocks = new BlockInfoContiguous[n];
|
|
|
|
- System.arraycopy(blocks, 0, newBlocks, 0, n);
|
|
|
|
|
|
+ newBlocks = new BlockInfo[n];
|
|
|
|
+ System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
|
|
}
|
|
}
|
|
// set new blocks
|
|
// set new blocks
|
|
- setContiguousBlocks(newBlocks);
|
|
|
|
|
|
+ setBlocks(newBlocks);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1059,10 +980,11 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
* snapshots. Since we do not support truncation with striped blocks,
|
|
* snapshots. Since we do not support truncation with striped blocks,
|
|
* we only need to handle contiguous blocks here.
|
|
* we only need to handle contiguous blocks here.
|
|
*/
|
|
*/
|
|
- public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
|
|
|
|
|
|
+ public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
|
|
BlocksMapUpdateInfo collectedBlocks) {
|
|
BlocksMapUpdateInfo collectedBlocks) {
|
|
- BlockInfoContiguous[] oldBlocks = this.blocks;
|
|
|
|
- if (snapshotBlocks == null || oldBlocks == null)
|
|
|
|
|
|
+ Preconditions.checkState(!isStriped());
|
|
|
|
+ BlockInfo[] oldBlocks = getBlocks();
|
|
|
|
+ if(snapshotBlocks == null || oldBlocks == null)
|
|
return;
|
|
return;
|
|
// Skip blocks in common between the file and the snapshot
|
|
// Skip blocks in common between the file and the snapshot
|
|
int n = 0;
|
|
int n = 0;
|
|
@@ -1070,7 +992,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
oldBlocks[n] == snapshotBlocks[n]) {
|
|
oldBlocks[n] == snapshotBlocks[n]) {
|
|
n++;
|
|
n++;
|
|
}
|
|
}
|
|
- truncateContiguousBlocks(n);
|
|
|
|
|
|
+ truncateBlocksTo(n);
|
|
// Collect the remaining blocks of the file
|
|
// Collect the remaining blocks of the file
|
|
while(n < oldBlocks.length) {
|
|
while(n < oldBlocks.length) {
|
|
collectedBlocks.addDeleteBlock(oldBlocks[n++]);
|
|
collectedBlocks.addDeleteBlock(oldBlocks[n++]);
|
|
@@ -1085,7 +1007,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
|
|
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
|
|
if(sf == null)
|
|
if(sf == null)
|
|
return;
|
|
return;
|
|
- BlockInfoContiguous[] snapshotBlocks =
|
|
|
|
|
|
+ BlockInfo[] snapshotBlocks =
|
|
getDiffs().findEarlierSnapshotBlocks(snapshotId);
|
|
getDiffs().findEarlierSnapshotBlocks(snapshotId);
|
|
if(snapshotBlocks == null)
|
|
if(snapshotBlocks == null)
|
|
return;
|
|
return;
|
|
@@ -1099,23 +1021,14 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
/**
|
|
/**
|
|
* @return true if the block is contained in a snapshot or false otherwise.
|
|
* @return true if the block is contained in a snapshot or false otherwise.
|
|
*/
|
|
*/
|
|
- boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
|
|
|
|
|
|
+ boolean isBlockInLatestSnapshot(BlockInfo block) {
|
|
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
|
|
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
|
|
if (sf == null || sf.getDiffs() == null) {
|
|
if (sf == null || sf.getDiffs() == null) {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
- BlockInfoContiguous[] snapshotBlocks = getDiffs()
|
|
|
|
|
|
+ BlockInfo[] snapshotBlocks = getDiffs()
|
|
.findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
|
|
.findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
|
|
return snapshotBlocks != null &&
|
|
return snapshotBlocks != null &&
|
|
Arrays.asList(snapshotBlocks).contains(block);
|
|
Arrays.asList(snapshotBlocks).contains(block);
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * @return true if the file is in the striping layout.
|
|
|
|
- */
|
|
|
|
- @VisibleForTesting
|
|
|
|
- @Override
|
|
|
|
- public boolean isStriped() {
|
|
|
|
- return getStripedBlocksFeature() != null;
|
|
|
|
- }
|
|
|
|
}
|
|
}
|