|
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
|
|
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.BlockInfoStripedUnderConstruction;
|
|
|
|
+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;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
|
|
@@ -295,7 +296,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
* Remove a block from the block list. This block should be
|
|
* Remove a block from the block list. This block should be
|
|
* the last one on the list.
|
|
* the last one on the list.
|
|
*/
|
|
*/
|
|
- BlockInfoContiguousUnderConstruction 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();
|
|
FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
|
|
@@ -317,7 +318,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
return uc;
|
|
return uc;
|
|
} else {
|
|
} else {
|
|
assert hasNoContiguousBlock();
|
|
assert hasNoContiguousBlock();
|
|
- return null;
|
|
|
|
|
|
+ return sb.removeLastBlock(oldblock);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -676,7 +677,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
final long ssDeltaNoReplication;
|
|
final long ssDeltaNoReplication;
|
|
short replication;
|
|
short replication;
|
|
if (isStriped()) {
|
|
if (isStriped()) {
|
|
- return computeQuotaUsageWithStriped(bsps, counts);
|
|
|
|
|
|
+ return computeQuotaUsageWithStriped(bsp, counts);
|
|
}
|
|
}
|
|
|
|
|
|
if (last < lastSnapshotId) {
|
|
if (last < lastSnapshotId) {
|
|
@@ -702,11 +703,15 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Compute quota of striped file
|
|
|
|
|
|
+ * Compute quota of striped file. Note that currently EC files do not support
|
|
|
|
+ * append/hflush/hsync, thus the file length recorded in snapshots should be
|
|
|
|
+ * the same with the current file length.
|
|
*/
|
|
*/
|
|
public final QuotaCounts computeQuotaUsageWithStriped(
|
|
public final QuotaCounts computeQuotaUsageWithStriped(
|
|
- BlockStoragePolicySuite bsps, QuotaCounts counts) {
|
|
|
|
- return null;
|
|
|
|
|
|
+ BlockStoragePolicy bsp, QuotaCounts counts) {
|
|
|
|
+ counts.addNameSpace(1);
|
|
|
|
+ counts.add(storagespaceConsumed(bsp));
|
|
|
|
+ return counts;
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -828,21 +833,44 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
* Use preferred block size for the last block if it is under construction.
|
|
* Use preferred block size for the last block if it is under construction.
|
|
*/
|
|
*/
|
|
public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
|
|
public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
|
|
- QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
|
|
if (isStriped()) {
|
|
if (isStriped()) {
|
|
- return storagespaceConsumedWithStriped(bsp);
|
|
|
|
|
|
+ return storagespaceConsumedWithStriped();
|
|
} else {
|
|
} else {
|
|
return storagespaceConsumedWithReplication(bsp);
|
|
return storagespaceConsumedWithReplication(bsp);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public final QuotaCounts storagespaceConsumedWithStriped(
|
|
|
|
- BlockStoragePolicy bsp) {
|
|
|
|
- return null;
|
|
|
|
|
|
+ // TODO: support EC with heterogeneous storage
|
|
|
|
+ public final QuotaCounts storagespaceConsumedWithStriped() {
|
|
|
|
+ QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
|
|
+ BlockInfo[] blockInfos = getBlocks();
|
|
|
|
+ if (blockInfos == null || blockInfos.length == 0) {
|
|
|
|
+ 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();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ counts.addStorageSpace(size);
|
|
|
|
+ return counts;
|
|
}
|
|
}
|
|
|
|
|
|
public final QuotaCounts storagespaceConsumedWithReplication(
|
|
public final QuotaCounts storagespaceConsumedWithReplication(
|
|
- BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
|
|
|
|
+ BlockStoragePolicy bsp) {
|
|
|
|
+ QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
final Iterable<BlockInfo> blocks;
|
|
final Iterable<BlockInfo> blocks;
|
|
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
|
|
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
|
|
if (sf == null) {
|
|
if (sf == null) {
|
|
@@ -965,6 +993,7 @@ public class INodeFile extends INodeWithAdditionalFields
|
|
/**
|
|
/**
|
|
* compute the quota usage change for a truncate op
|
|
* compute the quota usage change for a truncate op
|
|
* @param newLength the length for truncation
|
|
* @param newLength the length for truncation
|
|
|
|
+ * TODO: properly handle striped blocks (HDFS-7622)
|
|
**/
|
|
**/
|
|
void computeQuotaDeltaForTruncate(
|
|
void computeQuotaDeltaForTruncate(
|
|
long newLength, BlockStoragePolicy bsps,
|
|
long newLength, BlockStoragePolicy bsps,
|