HDFS-10759. Change fsimage bool isStriped from boolean to an enum. Contributed by Ewan Higgs.

This commit is contained in:
Andrew Wang 2017-01-18 13:31:05 -08:00
parent 8a0fa0f7e8
commit a2a5d7b5bc
31 changed files with 418 additions and 109 deletions

View File

@ -0,0 +1,57 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
/**
* Type of a block. Previously, all blocks were replicated (contiguous).
* Then Erasure Coded blocks (striped) were implemented.
*
* BlockTypes are currently defined by the highest bit in the block id. If
* this bit is set, then the block is striped.
*
* Further extensions may claim the second bit s.t. the highest two bits are
* set. e.g.
* 0b00 == contiguous
* 0b10 == striped
* 0b11 == possible further extension block type.
*/
public enum BlockType {
//! Replicated block.
CONTIGUOUS,
//! Erasure Coded Block
STRIPED;
// BLOCK_ID_MASK is the union of all masks.
static final long BLOCK_ID_MASK = 1L << 63;
// BLOCK_ID_MASK_STRIPED is the mask for striped blocks.
static final long BLOCK_ID_MASK_STRIPED = 1L << 63;
/**
* Parse a BlockId to find the BlockType
* Note: the old block id generation algorithm was based on a pseudo random
* number generator, so there may be legacy blocks that make this conversion
* unreliable.
*/
public static BlockType fromBlockId(long blockId) {
long blockType = blockId & BLOCK_ID_MASK;
if(blockType == BLOCK_ID_MASK_STRIPED) {
return STRIPED;
}
return CONTIGUOUS;
}
}

View File

@ -211,4 +211,8 @@ public class LocatedBlock {
public boolean isStriped() {
return false;
}
public BlockType getBlockType() {
return BlockType.CONTIGUOUS;
}
}

View File

@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.inotify.EventBatch;
import org.apache.hadoop.hdfs.inotify.EventBatchList;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
@ -122,6 +123,7 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZon
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
@ -1775,6 +1777,24 @@ public class PBHelperClient {
return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
}
public static BlockTypeProto convert(BlockType blockType) {
switch (blockType) {
case CONTIGUOUS: return BlockTypeProto.CONTIGUOUS;
case STRIPED: return BlockTypeProto.STRIPED;
default:
throw new IllegalArgumentException("Unexpected block type: " + blockType);
}
}
public static BlockType convert(BlockTypeProto blockType) {
switch (blockType.getNumber()) {
case BlockTypeProto.CONTIGUOUS_VALUE: return BlockType.CONTIGUOUS;
case BlockTypeProto.STRIPED_VALUE: return BlockType.STRIPED;
default:
throw new IllegalArgumentException("Unexpected block type: " + blockType);
}
}
static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
if (di == null) return null;
DatanodeInfo[] result = new DatanodeInfo[di.length];

View File

@ -197,6 +197,14 @@ enum StorageTypeProto {
RAM_DISK = 4;
}
/**
* Types of recognized blocks.
*/
enum BlockTypeProto {
CONTIGUOUS = 0;
STRIPED = 1;
}
/**
* A list of storage types.
*/

View File

@ -0,0 +1,61 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocol;
import org.junit.Test;
import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
import static org.junit.Assert.*;
/**
* Test the BlockType class.
*/
public class TestBlockType {
@Test
public void testGetBlockType() throws Exception {
assertEquals(BlockType.fromBlockId(0x0000000000000000L), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x1000000000000000L), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x2000000000000000L), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x4000000000000000L), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x7000000000000000L), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x00000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x10000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x20000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x40000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x70000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x70000000ffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x0fffffffffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x1fffffffffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x2fffffffffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x4fffffffffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x7fffffffffffffffL), CONTIGUOUS);
assertEquals(BlockType.fromBlockId(0x8000000000000000L), STRIPED);
assertEquals(BlockType.fromBlockId(0x9000000000000000L), STRIPED);
assertEquals(BlockType.fromBlockId(0xa000000000000000L), STRIPED);
assertEquals(BlockType.fromBlockId(0xf000000000000000L), STRIPED);
assertEquals(BlockType.fromBlockId(0x80000000ffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0x90000000ffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0xa0000000ffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0xf0000000ffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0x8fffffffffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0x9fffffffffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0xafffffffffffffffL), STRIPED);
assertEquals(BlockType.fromBlockId(0xffffffffffffffffL), STRIPED);
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.balancer;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
import java.io.IOException;
import java.io.PrintStream;
@ -220,7 +221,7 @@ public class Balancer {
) throws UnsupportedActionException {
BlockPlacementPolicies placementPolicies =
new BlockPlacementPolicies(conf, null, null, null);
if (!(placementPolicies.getPolicy(false) instanceof
if (!(placementPolicies.getPolicy(CONTIGUOUS) instanceof
BlockPlacementPolicyDefault)) {
throw new UnsupportedActionException(
"Balancer without BlockPlacementPolicyDefault");

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@ -1274,7 +1275,7 @@ public class Dispatcher {
}
datanodeInfos.add(target.getDatanodeInfo());
}
return placementPolicies.getPolicy(false).isMovable(
return placementPolicies.getPolicy(BlockType.CONTIGUOUS).isMovable(
datanodeInfos, source.getDatanodeInfo(), target.getDatanodeInfo());
}

View File

@ -20,12 +20,15 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import java.io.IOException;
import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
/**
* BlockIdManager allocates the generation stamps and the block ID. The
* {@see FSNamesystem} is responsible for persisting the allocations in the
@ -207,9 +210,14 @@ public class BlockIdManager {
/**
* Increments, logs and then returns the block ID
*/
long nextBlockId(boolean isStriped) {
return isStriped ? blockGroupIdGenerator.nextValue() :
blockIdGenerator.nextValue();
long nextBlockId(BlockType blockType) {
switch(blockType) {
case CONTIGUOUS: return blockIdGenerator.nextValue();
case STRIPED: return blockGroupIdGenerator.nextValue();
default:
throw new IllegalArgumentException(
"nextBlockId called with an unsupported BlockType");
}
}
boolean isGenStampInFuture(Block block) {
@ -230,7 +238,7 @@ public class BlockIdManager {
}
public static boolean isStripedBlockID(long id) {
return id < 0;
return BlockType.fromBlockId(id) == STRIPED;
}
/**

View File

@ -25,6 +25,7 @@ import java.util.NoSuchElementException;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.util.LightWeightGSet;
@ -175,6 +176,8 @@ public abstract class BlockInfo extends Block
public abstract boolean isStriped();
public abstract BlockType getBlockType();
/** @return true if there is no datanode storage associated with the block */
abstract boolean hasNoStorage();
@ -262,11 +265,11 @@ public abstract class BlockInfo extends Block
DatanodeStorageInfo[] targets) {
if (isComplete()) {
uc = new BlockUnderConstructionFeature(this, s, targets,
this.isStriped());
this.getBlockType());
} else {
// the block is already under construction
uc.setBlockUCState(s);
uc.setExpectedLocations(this, targets, this.isStriped());
uc.setExpectedLocations(this, targets, this.getBlockType());
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
/**
* Subclass of {@link BlockInfo}, used for a block with replication scheme.
@ -92,6 +93,11 @@ public class BlockInfoContiguous extends BlockInfo {
return false;
}
@Override
public BlockType getBlockType() {
return BlockType.CONTIGUOUS;
}
@Override
final boolean hasNoStorage() {
return getStorageInfo(0) == null;

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@ -208,6 +209,11 @@ public class BlockInfoStriped extends BlockInfo {
return true;
}
@Override
public BlockType getBlockType() {
return BlockType.STRIPED;
}
@Override
public int numNodes() {
assert this.storages != null : "BlockInfo is not initialized";

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
import static org.apache.hadoop.util.ExitUtil.terminate;
import java.io.IOException;
@ -60,6 +62,7 @@ import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -642,7 +645,7 @@ public class BlockManager implements BlockStatsMXBean {
@VisibleForTesting
public BlockPlacementPolicy getBlockPlacementPolicy() {
return placementPolicies.getPolicy(false);
return placementPolicies.getPolicy(CONTIGUOUS);
}
/** Dump meta data to out. */
@ -784,10 +787,13 @@ public class BlockManager implements BlockStatsMXBean {
}
public int getDefaultStorageNum(BlockInfo block) {
if (block.isStriped()) {
return ((BlockInfoStriped) block).getRealTotalBlockNum();
} else {
return defaultReplication;
switch (block.getBlockType()) {
case STRIPED: return ((BlockInfoStriped) block).getRealTotalBlockNum();
case CONTIGUOUS: return defaultReplication;
default:
throw new IllegalArgumentException(
"getDefaultStorageNum called with unknown BlockType: "
+ block.getBlockType());
}
}
@ -796,10 +802,13 @@ public class BlockManager implements BlockStatsMXBean {
}
public short getMinStorageNum(BlockInfo block) {
if (block.isStriped()) {
return ((BlockInfoStriped) block).getRealDataBlockNum();
} else {
return minReplication;
switch(block.getBlockType()) {
case STRIPED: return ((BlockInfoStriped) block).getRealDataBlockNum();
case CONTIGUOUS: return minReplication;
default:
throw new IllegalArgumentException(
"getMinStorageNum called with unknown BlockType: "
+ block.getBlockType());
}
}
@ -1723,7 +1732,7 @@ public class BlockManager implements BlockStatsMXBean {
// It is costly to extract the filename for which chooseTargets is called,
// so for now we pass in the block collection itself.
final BlockPlacementPolicy placementPolicy =
placementPolicies.getPolicy(rw.getBlock().isStriped());
placementPolicies.getPolicy(rw.getBlock().getBlockType());
rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
}
@ -1928,9 +1937,9 @@ public class BlockManager implements BlockStatsMXBean {
/** Choose target for WebHDFS redirection. */
public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode,
Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
blocksize, storagePolicySuite.getDefaultPolicy(), null);
return placementPolicies.getPolicy(CONTIGUOUS).chooseTarget(src, 1,
clientnode, Collections.<DatanodeStorageInfo>emptyList(), false,
excludes, blocksize, storagePolicySuite.getDefaultPolicy(), null);
}
/** Choose target for getting additional datanodes for an existing pipeline. */
@ -1941,9 +1950,11 @@ public class BlockManager implements BlockStatsMXBean {
Set<Node> excludes,
long blocksize,
byte storagePolicyID,
boolean isStriped) {
final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
BlockType blockType) {
final BlockStoragePolicy storagePolicy =
storagePolicySuite.getPolicy(storagePolicyID);
final BlockPlacementPolicy blockplacement =
placementPolicies.getPolicy(blockType);
return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
chosen, true, excludes, blocksize, storagePolicy, null);
}
@ -1962,12 +1973,14 @@ public class BlockManager implements BlockStatsMXBean {
final long blocksize,
final List<String> favoredNodes,
final byte storagePolicyID,
final boolean isStriped,
final BlockType blockType,
final EnumSet<AddBlockFlag> flags) throws IOException {
List<DatanodeDescriptor> favoredDatanodeDescriptors =
getDatanodeDescriptors(favoredNodes);
final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
final BlockStoragePolicy storagePolicy =
storagePolicySuite.getPolicy(storagePolicyID);
final BlockPlacementPolicy blockplacement =
placementPolicies.getPolicy(blockType);
final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
numOfReplicas, client, excludedNodes, blocksize,
favoredDatanodeDescriptors, storagePolicy, flags);
@ -3428,7 +3441,7 @@ public class BlockManager implements BlockStatsMXBean {
final Collection<DatanodeStorageInfo> nonExcess, BlockInfo storedBlock,
short replication, DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint, List<StorageType> excessTypes) {
BlockPlacementPolicy replicator = placementPolicies.getPolicy(false);
BlockPlacementPolicy replicator = placementPolicies.getPolicy(CONTIGUOUS);
List<DatanodeStorageInfo> replicasToDelete = replicator
.chooseReplicasToDelete(nonExcess, nonExcess, replication, excessTypes,
addedNode, delNodeHint);
@ -3490,7 +3503,7 @@ public class BlockManager implements BlockStatsMXBean {
return;
}
BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true);
BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(STRIPED);
// for each duplicated index, delete some replicas until only one left
for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0;
targetIndex = duplicated.nextSetBit(targetIndex + 1)) {
@ -4145,9 +4158,10 @@ public class BlockManager implements BlockStatsMXBean {
}
}
DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]);
BlockType blockType = storedBlock.getBlockType();
BlockPlacementPolicy placementPolicy = placementPolicies
.getPolicy(storedBlock.isStriped());
int numReplicas = storedBlock.isStriped() ? ((BlockInfoStriped) storedBlock)
.getPolicy(blockType);
int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock)
.getRealDataBlockNum() : storedBlock.getReplication();
return placementPolicy.verifyBlockPlacement(locs, numReplicas)
.isPlacementPolicySatisfied();
@ -4685,8 +4699,8 @@ public class BlockManager implements BlockStatsMXBean {
return blockIdManager.isLegacyBlock(block);
}
public long nextBlockId(boolean isStriped) {
return blockIdManager.nextBlockId(isStriped);
public long nextBlockId(BlockType blockType) {
return blockIdManager.nextBlockId(blockType);
}
boolean isGenStampInFuture(Block block) {

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.util.ReflectionUtils;
@ -46,11 +47,13 @@ public class BlockPlacementPolicies{
ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
}
public BlockPlacementPolicy getPolicy(boolean isStriped){
if (isStriped) {
return ecPolicy;
} else {
return replicationPolicy;
public BlockPlacementPolicy getPolicy(BlockType blockType){
switch (blockType) {
case CONTIGUOUS: return replicationPolicy;
case STRIPED: return ecPolicy;
default:
throw new IllegalArgumentException(
"getPolicy received a BlockType that isn't supported.");
}
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.blockmanagement;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
@ -60,16 +61,16 @@ public class BlockUnderConstructionFeature {
private Block truncateBlock;
public BlockUnderConstructionFeature(Block blk,
BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
BlockUCState state, DatanodeStorageInfo[] targets, BlockType blockType) {
assert getBlockUCState() != COMPLETE :
"BlockUnderConstructionFeature cannot be in COMPLETE state";
this.blockUCState = state;
setExpectedLocations(blk, targets, isStriped);
setExpectedLocations(blk, targets, blockType);
}
/** Set expected locations */
public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
boolean isStriped) {
BlockType blockType) {
if (targets == null) {
return;
}
@ -86,7 +87,7 @@ public class BlockUnderConstructionFeature {
if (targets[i] != null) {
// when creating a new striped block we simply sequentially assign block
// index to each storage
Block replicaBlock = isStriped ?
Block replicaBlock = blockType == BlockType.STRIPED ?
new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
block;
replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
@ -227,7 +228,8 @@ final class FSDirTruncateOp {
boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
if (newBlock == null) {
newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false)
newBlock = (shouldCopyOnTruncate) ?
fsn.createNewBlock(BlockType.CONTIGUOUS)
: new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
fsn.nextGenerationStamp(fsn.getBlockManager().isLegacyBlock(
oldBlock)));

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@ -119,7 +120,7 @@ class FSDirWriteFileOp {
FSNamesystem fsn = fsd.getFSNamesystem();
final INodeFile file = fsn.checkLease(iip, holder, fileId);
Preconditions.checkState(file.isUnderConstruction());
if (file.isStriped()) {
if (file.getBlockType() == BlockType.STRIPED) {
return; // do not abandon block for striped file
}
@ -162,7 +163,7 @@ class FSDirWriteFileOp {
final short numTargets;
final byte storagePolicyID;
String clientMachine;
final boolean isStriped;
final BlockType blockType;
INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
@ -186,9 +187,9 @@ class FSDirWriteFileOp {
blockSize = pendingFile.getPreferredBlockSize();
clientMachine = pendingFile.getFileUnderConstructionFeature()
.getClientMachine();
isStriped = pendingFile.isStriped();
blockType = pendingFile.getBlockType();
ErasureCodingPolicy ecPolicy = null;
if (isStriped) {
if (blockType == BlockType.STRIPED) {
ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
+ ecPolicy.getSchema().getNumParityUnits());
@ -197,7 +198,7 @@ class FSDirWriteFileOp {
}
storagePolicyID = pendingFile.getStoragePolicyID();
return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
clientMachine, isStriped);
clientMachine, blockType);
}
static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@ -237,7 +238,7 @@ class FSDirWriteFileOp {
// add new chosen targets to already allocated block and return
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
lastBlockInFile, targets, pendingFile.isStriped());
lastBlockInFile, targets, pendingFile.getBlockType());
offset = pendingFile.computeFileSize();
return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
}
@ -248,11 +249,11 @@ class FSDirWriteFileOp {
ExtendedBlock.getLocalBlock(previous));
// allocate new block, record block locations in INode.
final boolean isStriped = pendingFile.isStriped();
final BlockType blockType = pendingFile.getBlockType();
// allocate new block, record block locations in INode.
Block newBlock = fsn.createNewBlock(isStriped);
Block newBlock = fsn.createNewBlock(blockType);
INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped);
saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, blockType);
persistNewBlock(fsn, src, pendingFile);
offset = pendingFile.computeFileSize();
@ -282,7 +283,7 @@ class FSDirWriteFileOp {
return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
excludedNodesSet, r.blockSize,
favoredNodesList, r.storagePolicyID,
r.isStriped, flags);
r.blockType, flags);
}
/**
@ -420,14 +421,16 @@ class FSDirWriteFileOp {
if (ecPolicy != null) {
replication = ecPolicy.getId();
}
final BlockType blockType = ecPolicy != null?
BlockType.STRIPED : BlockType.CONTIGUOUS;
if (underConstruction) {
newNode = newINodeFile(id, permissions, modificationTime,
modificationTime, replication, preferredBlockSize, storagePolicyId,
ecPolicy != null);
blockType);
newNode.toUnderConstruction(clientName, clientMachine);
} else {
newNode = newINodeFile(id, permissions, modificationTime, atime,
replication, preferredBlockSize, storagePolicyId, ecPolicy != null);
replication, preferredBlockSize, storagePolicyId, blockType);
}
newNode.setLocalName(localName);
INodesInPath iip = fsd.addINode(existing, newNode,
@ -459,7 +462,7 @@ class FSDirWriteFileOp {
*/
private static BlockInfo addBlock(FSDirectory fsd, String path,
INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
boolean isStriped) throws IOException {
BlockType blockType) throws IOException {
fsd.writeLock();
try {
final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@ -467,7 +470,7 @@ class FSDirWriteFileOp {
// associate new last block for the file
final BlockInfo blockInfo;
if (isStriped) {
if (blockType == BlockType.STRIPED) {
ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
fsd.getFSNamesystem(), inodesInPath);
short numDataUnits = (short) ecPolicy.getNumDataUnits();
@ -525,8 +528,10 @@ class FSDirWriteFileOp {
if (ecPolicy != null) {
replication = ecPolicy.getId();
}
final BlockType blockType = ecPolicy != null?
BlockType.STRIPED : BlockType.CONTIGUOUS;
INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
modTime, modTime, replication, preferredBlockSize, blockType);
newNode.setLocalName(localName);
newNode.toUnderConstruction(clientName, clientMachine);
newiip = fsd.addINode(existing, newNode, permissions.getPermission());
@ -698,17 +703,17 @@ class FSDirWriteFileOp {
private static INodeFile newINodeFile(
long id, PermissionStatus permissions, long mtime, long atime,
short replication, long preferredBlockSize, byte storagePolicyId,
boolean isStriped) {
BlockType blockType) {
return new INodeFile(id, null, permissions, mtime, atime,
BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
storagePolicyId, isStriped);
storagePolicyId, blockType);
}
private static INodeFile newINodeFile(long id, PermissionStatus permissions,
long mtime, long atime, short replication, long preferredBlockSize,
boolean isStriped) {
BlockType blockType) {
return newINodeFile(id, permissions, mtime, atime, replication,
preferredBlockSize, (byte)0, isStriped);
preferredBlockSize, (byte)0, blockType);
}
/**
@ -738,10 +743,10 @@ class FSDirWriteFileOp {
*/
private static void saveAllocatedBlock(FSNamesystem fsn, String src,
INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
boolean isStriped) throws IOException {
BlockType blockType) throws IOException {
assert fsn.hasWriteLock();
BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
isStriped);
blockType);
logAllocatedBlock(src, b);
DatanodeStorageInfo.incrementBlocksScheduled(targets);
}
@ -808,16 +813,16 @@ class FSDirWriteFileOp {
final int numTargets;
final byte storagePolicyID;
final String clientMachine;
final boolean isStriped;
final BlockType blockType;
ValidateAddBlockResult(
long blockSize, int numTargets, byte storagePolicyID,
String clientMachine, boolean isStriped) {
String clientMachine, BlockType blockType) {
this.blockSize = blockSize;
this.numTargets = numTargets;
this.storagePolicyID = storagePolicyID;
this.clientMachine = clientMachine;
this.isStriped = isStriped;
this.blockType = blockType;
}
}
}

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -896,7 +897,8 @@ public class FSImageFormat {
final long preferredBlockSize = in.readLong();
return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
accessTime, replication, preferredBlockSize, (byte) 0, null, false);
accessTime, replication, preferredBlockSize, (byte) 0, null,
BlockType.CONTIGUOUS);
}
public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@ -329,14 +330,14 @@ public final class FSImageFormatPBINode {
INodeSection.INodeFile f = n.getFile();
List<BlockProto> bp = f.getBlocksList();
short replication = (short) f.getReplication();
boolean isStriped = f.getIsStriped();
BlockType blockType = PBHelperClient.convert(f.getBlockType());
LoaderContext state = parent.getLoaderContext();
ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
BlockInfo[] blocks = new BlockInfo[bp.size()];
for (int i = 0; i < bp.size(); ++i) {
BlockProto b = bp.get(i);
if (isStriped) {
if (blockType == BlockType.STRIPED) {
blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
} else {
blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
@ -350,7 +351,7 @@ public final class FSImageFormatPBINode {
final INodeFile file = new INodeFile(n.getId(),
n.getName().toByteArray(), permissions, f.getModificationTime(),
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
(byte)f.getStoragePolicyID(), isStriped);
(byte)f.getStoragePolicyID(), blockType);
if (f.hasAcl()) {
int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
@ -373,7 +374,7 @@ public final class FSImageFormatPBINode {
BlockInfo lastBlk = file.getLastBlock();
// replace the last block of file
final BlockInfo ucBlk;
if (isStriped) {
if (blockType == BlockType.STRIPED) {
BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
ucBlk = new BlockInfoStriped(striped, ecPolicy);
} else {
@ -502,7 +503,7 @@ public final class FSImageFormatPBINode {
.setPreferredBlockSize(file.getPreferredBlockSize())
.setReplication(file.getFileReplication())
.setStoragePolicyID(file.getLocalStoragePolicyID())
.setIsStriped(file.isStriped());
.setBlockType(PBHelperClient.convert(file.getBlockType()));
AclFeature f = file.getAclFeature();
if (f != null) {

View File

@ -207,6 +207,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@ -2541,7 +2542,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final long preferredblocksize;
final byte storagePolicyID;
final List<DatanodeStorageInfo> chosen;
final boolean isStriped;
final BlockType blockType;
checkOperation(OperationCategory.READ);
FSPermissionChecker pc = getPermissionChecker();
readLock();
@ -2558,7 +2559,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
preferredblocksize = file.getPreferredBlockSize();
storagePolicyID = file.getStoragePolicyID();
isStriped = file.isStriped();
blockType = file.getBlockType();
//find datanode storages
final DatanodeManager dm = blockManager.getDatanodeManager();
@ -2576,7 +2577,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// choose new datanodes.
final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
src, numAdditionalNodes, clientnode, chosen,
excludes, preferredblocksize, storagePolicyID, isStriped);
excludes, preferredblocksize, storagePolicyID, blockType);
final LocatedBlock lb = BlockManager.newLocatedBlock(
blk, targets, -1, false);
blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
@ -2676,11 +2677,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
/**
* Create new block with a unique block id and a new generation stamp.
* @param isStriped is the file under striping or contiguous layout?
* @param blockType is the file under striping or contiguous layout?
*/
Block createNewBlock(boolean isStriped) throws IOException {
Block createNewBlock(BlockType blockType) throws IOException {
assert hasWriteLock();
Block b = new Block(nextBlockId(isStriped), 0, 0);
Block b = new Block(nextBlockId(blockType), 0, 0);
// Increment the generation stamp for every new block.
b.setGenerationStamp(nextGenerationStamp(false));
return b;
@ -3194,7 +3195,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// setup the last block locations from the blockManager if not known
if (uc.getNumExpectedLocations() == 0) {
uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
lastBlock.isStriped());
lastBlock.getBlockType());
}
if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
@ -4716,12 +4717,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
/**
* Increments, logs and then returns the block ID
* @param isStriped is the file under striping or contiguous layout?
* @param blockType is the file under striping or contiguous layout?
*/
private long nextBlockId(boolean isStriped) throws IOException {
private long nextBlockId(BlockType blockType) throws IOException {
assert hasWriteLock();
checkNameNodeSafeMode("Cannot get next block ID");
final long blockId = blockManager.nextBlockId(isStriped);
final long blockId = blockManager.nextBlockId(blockType);
getEditLog().logAllocateBlockId(blockId);
// NB: callers sync the log
return blockId;
@ -4936,7 +4937,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
"src=%s, oldBlock=%s, newBlock=%s, clientName=%s",
src, oldBlock, newBlock, clientName);
lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
storages, lastBlock.isStriped());
storages, lastBlock.getBlockType());
FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);
}

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
@ -43,6 +45,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@ -93,6 +96,10 @@ public class INodeFile extends INodeWithAdditionalFields
* stores the EC policy ID, and in the future, we may further divide these
* 11 bits to store both the EC policy ID and replication factor for erasure
* coded blocks. The layout of this section is demonstrated as below.
*
* Another possible future extension is for future block types, in which case
* the 'Replica or EC' bit may be extended into the 11 bit field.
*
* +---------------+-------------------------------+
* | 1 bit | 11 bit |
* +---------------+-------------------------------+
@ -144,13 +151,29 @@ public class INodeFile extends INodeWithAdditionalFields
return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
}
// Union of all the block type masks. Currently there is only
// BLOCK_TYPE_MASK_STRIPED
static final long BLOCK_TYPE_MASK = 1 << 11;
// Mask to determine if the block type is striped.
static final long BLOCK_TYPE_MASK_STRIPED = 1 << 11;
static boolean isStriped(long header) {
long layoutRedundancy = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.retrieve(header);
return (layoutRedundancy & (1 << 11)) != 0;
return (layoutRedundancy & BLOCK_TYPE_MASK) != 0;
}
static BlockType getBlockType(long header) {
long layoutRedundancy = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.retrieve(header);
long blockType = layoutRedundancy & BLOCK_TYPE_MASK;
if (blockType == BLOCK_TYPE_MASK_STRIPED) {
return STRIPED;
} else {
return CONTIGUOUS;
}
}
static long toLong(long preferredBlockSize, short replication,
boolean isStriped, byte storagePolicyID) {
BlockType blockType, byte storagePolicyID) {
Preconditions.checkArgument(replication >= 0 &&
replication <= MAX_REDUNDANCY);
long h = 0;
@ -161,8 +184,8 @@ public class INodeFile extends INodeWithAdditionalFields
// For erasure coded files, replication is used to store ec policy id
// TODO: this is hacky. Add some utility to generate the layoutRedundancy
long layoutRedundancy = 0;
if (isStriped) {
layoutRedundancy |= 1 << 11;
if (blockType == STRIPED) {
layoutRedundancy |= BLOCK_TYPE_MASK_STRIPED;
}
layoutRedundancy |= replication;
h = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.combine(layoutRedundancy, h);
@ -180,18 +203,18 @@ public class INodeFile extends INodeWithAdditionalFields
long atime, BlockInfo[] blklist, short replication,
long preferredBlockSize) {
this(id, name, permissions, mtime, atime, blklist, replication,
preferredBlockSize, (byte) 0, false);
preferredBlockSize, (byte) 0, CONTIGUOUS);
}
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfo[] blklist, short replication,
long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
long preferredBlockSize, byte storagePolicyID, BlockType blockType) {
super(id, name, permissions, mtime, atime);
header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
storagePolicyID);
if (blklist != null && blklist.length > 0) {
for (BlockInfo b : blklist) {
Preconditions.checkArgument(b.isStriped() == isStriped);
Preconditions.checkArgument(b.getBlockType() == blockType);
}
}
setBlocks(blklist);
@ -554,6 +577,15 @@ public class INodeFile extends INodeWithAdditionalFields
return HeaderFormat.isStriped(header);
}
/**
* @return The type of the INodeFile based on block id.
*/
@VisibleForTesting
@Override
public BlockType getBlockType() {
return HeaderFormat.getBlockType(header);
}
@Override // INodeFileAttributes
public long getHeaderLong() {
return header;

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
/**
@ -32,6 +33,9 @@ public interface INodeFileAttributes extends INodeAttributes {
/** @return whether the file is striped (instead of contiguous) */
boolean isStriped();
/** @return whether the file is striped (instead of contiguous) */
BlockType getBlockType();
/** @return the ID of the ErasureCodingPolicy */
byte getErasureCodingPolicyID();
@ -53,10 +57,10 @@ public interface INodeFileAttributes extends INodeAttributes {
public SnapshotCopy(byte[] name, PermissionStatus permissions,
AclFeature aclFeature, long modificationTime, long accessTime,
short replication, long preferredBlockSize,
byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
byte storagePolicyID, XAttrFeature xAttrsFeature, BlockType blockType) {
super(name, permissions, aclFeature, modificationTime, accessTime,
xAttrsFeature);
header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
storagePolicyID);
}
@ -80,6 +84,11 @@ public interface INodeFileAttributes extends INodeAttributes {
return HeaderFormat.isStriped(header);
}
@Override
public BlockType getBlockType() {
return HeaderFormat.getBlockType(header);
}
@Override
public byte getErasureCodingPolicyID() {
if (isStriped()) {

View File

@ -721,7 +721,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
// count mis replicated blocks
BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(
lBlk.isStriped()).verifyBlockPlacement(lBlk.getLocations(),
lBlk.getBlockType()).verifyBlockPlacement(lBlk.getLocations(),
targetFileReplication);
if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
res.numMisReplicatedBlocks++;

View File

@ -237,7 +237,7 @@ public class FSImageFormatPBSnapshot {
fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
fileInPb.getPreferredBlockSize(),
(byte)fileInPb.getStoragePolicyID(), xAttrs,
fileInPb.getIsStriped());
PBHelperClient.convert(fileInPb.getBlockType()));
}
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

View File

@ -649,8 +649,20 @@ class OfflineImageReconstructor {
if (ival != null) {
bld.setStoragePolicyID(ival);
}
Boolean bval = node.removeChildBool(INODE_SECTION_IS_STRIPED);
bld.setIsStriped(bval);
String blockType = node.removeChildStr(INODE_SECTION_BLOCK_TYPE);
if(blockType != null) {
switch (blockType) {
case "CONTIGUOUS":
bld.setBlockType(HdfsProtos.BlockTypeProto.CONTIGUOUS);
break;
case "STRIPED":
bld.setBlockType(HdfsProtos.BlockTypeProto.STRIPED);
break;
default:
throw new IOException("INode XML found with unknown <blocktype> " +
blockType);
}
}
inodeBld.setFile(bld);
// Will check remaining keys and serialize in processINodeXml
}

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheD
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
@ -130,7 +131,7 @@ public final class PBImageXmlWriter {
public static final String INODE_SECTION_XATTRS = "xattrs";
public static final String INODE_SECTION_STORAGE_POLICY_ID =
"storagePolicyId";
public static final String INODE_SECTION_IS_STRIPED = "isStriped";
public static final String INODE_SECTION_BLOCK_TYPE = "blockType";
public static final String INODE_SECTION_NS_QUOTA = "nsquota";
public static final String INODE_SECTION_DS_QUOTA = "dsquota";
public static final String INODE_SECTION_TYPE_QUOTA = "typeQuota";
@ -492,8 +493,10 @@ public final class PBImageXmlWriter {
if (f.hasStoragePolicyID()) {
o(INODE_SECTION_STORAGE_POLICY_ID, f.getStoragePolicyID());
}
if (f.getIsStriped()) {
out.print("<" + INODE_SECTION_IS_STRIPED + "/>");
if (f.getBlockType() != BlockTypeProto.CONTIGUOUS) {
out.print("<" + INODE_SECTION_BLOCK_TYPE + ">");
o(SECTION_NAME, f.getBlockType().name());
out.print("</" + INODE_SECTION_BLOCK_TYPE + ">\n");
}
if (f.hasFileUC()) {

View File

@ -128,7 +128,7 @@ message INodeSection {
message XAttrFeatureProto {
repeated XAttrCompactProto xAttrs = 1;
}
message INodeFile {
optional uint32 replication = 1;
optional uint64 modificationTime = 2;
@ -140,7 +140,7 @@ message INodeSection {
optional AclFeatureProto acl = 8;
optional XAttrFeatureProto xAttrs = 9;
optional uint32 storagePolicyID = 10;
optional bool isStriped = 11;
optional BlockTypeProto blockType = 11;
}
message QuotaByStorageTypeEntryProto {

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecover
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
@ -199,6 +201,19 @@ public class TestPBHelper {
assertEquals(b, b2);
}
@Test
public void testConvertBlockType() {
BlockType bContiguous = BlockType.CONTIGUOUS;
BlockTypeProto bContiguousProto = PBHelperClient.convert(bContiguous);
BlockType bContiguous2 = PBHelperClient.convert(bContiguousProto);
assertEquals(bContiguous, bContiguous2);
BlockType bStriped = BlockType.STRIPED;
BlockTypeProto bStripedProto = PBHelperClient.convert(bStriped);
BlockType bStriped2 = PBHelperClient.convert(bStripedProto);
assertEquals(bStriped, bStriped2);
}
private static BlockWithLocations getBlockWithLocations(
int bid, boolean isStriped) {
final String[] datanodeUuids = {"dn1", "dn2", "dn3"};

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -167,7 +168,7 @@ public class TestDeadDatanode {
// part of the cluster anymore
DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
false, null);
BlockType.CONTIGUOUS, null);
for (DatanodeStorageInfo datanodeStorageInfo : results) {
assertFalse("Dead node should not be choosen", datanodeStorageInfo
.getDatanodeDescriptor().equals(clientNode));

View File

@ -33,9 +33,11 @@ import java.util.EnumSet;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.junit.Assert;
@ -58,6 +60,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils;
@ -160,7 +163,7 @@ public class TestFSImage {
short replication = testECPolicy.getId();
long preferredBlockSize = 128*1024*1024;
INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
blks, replication, preferredBlockSize, (byte) 0, true);
blks, replication, preferredBlockSize, (byte) 0, BlockType.STRIPED);
ByteArrayOutputStream bs = new ByteArrayOutputStream();
//construct StripedBlocks for the INode
@ -646,4 +649,14 @@ public class TestFSImage {
}
}
}
@Test
public void testBlockTypeProtoDefaultsToContiguous() throws Exception {
INodeSection.INodeFile.Builder builder = INodeSection.INodeFile
.newBuilder();
INodeSection.INodeFile inodeFile = builder.build();
BlockType defaultBlockType = PBHelperClient.convert(inodeFile
.getBlockType());
assertEquals(defaultBlockType, BlockType.CONTIGUOUS);
}
}

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertSame;
@ -102,9 +104,15 @@ public class TestINodeFile {
null, replication, preferredBlockSize);
}
INodeFile createStripedINodeFile(short replication, long preferredBlockSize) {
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, replication, preferredBlockSize,
HdfsConstants.WARM_STORAGE_POLICY_ID, STRIPED);
}
private static INodeFile createINodeFile(byte storagePolicyID) {
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, (short)3, 1024L, storagePolicyID, false);
null, (short)3, 1024L, storagePolicyID, CONTIGUOUS);
}
@Test
@ -215,14 +223,24 @@ public class TestINodeFile {
dir.addChild(inf);
assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
root.addChild(dir);
assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
assertEquals(Path.SEPARATOR, root.getFullPathName());
}
@Test
public void testGetBlockType() {
replication = 3;
preferredBlockSize = 128*1024*1024;
INodeFile inf = createINodeFile(replication, preferredBlockSize);
assertEquals(inf.getBlockType(), CONTIGUOUS);
INodeFile striped = createStripedINodeFile(replication, preferredBlockSize);
assertEquals(striped.getBlockType(), STRIPED);
}
/**
* FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
* replace the original INodeDirectory. Before HDFS-4243, the parent field of

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.protocol.BlockType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.junit.Assert;
import org.junit.Rule;
@ -75,7 +76,8 @@ public class TestStripedINodeFile {
private static INodeFile createStripedINodeFile() {
return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID,
BlockType.STRIPED);
}
@Test