diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e315db6702a..033451eb02c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -626,6 +626,9 @@ Release 2.8.0 - UNRELEASED
HDFS-7923. The DataNodes should rate-limit their full block reports by
asking the NN on heartbeat messages (cmccabe)
+ HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
+ (Zhe Zhang via wang)
+
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 02a1d0522de..f11a825c62d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -79,7 +79,7 @@ public interface BlockCollection {
* Convert the last block of the collection to an under-construction block
* and set the locations.
*/
- public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfo lastBlock,
+ public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeStorageInfo[] targets) throws IOException;
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index dea31c42f97..4cc2791e754 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -51,7 +51,7 @@ public abstract class BlockInfo extends Block
* per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
* bytes using the triplets.
*/
- protected Object[] triplets;
+ Object[] triplets;
/**
* Construct an entry for blocksmap
@@ -295,7 +295,7 @@ public abstract class BlockInfo extends Block
/**
* BlockInfo represents a block that is not being constructed.
* In order to start modifying the block, the BlockInfo should be converted
- * to {@link BlockInfoContiguousUnderConstruction}.
+ * to {@link BlockInfoUnderConstruction}.
* @return {@link BlockUCState#COMPLETE}
*/
public BlockUCState getBlockUCState() {
@@ -312,27 +312,29 @@ public abstract class BlockInfo extends Block
}
/**
- * Convert a complete block to an under construction block.
+ * Convert a block to an under construction block.
* @return BlockInfoUnderConstruction - an under construction block.
*/
- public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
+ public BlockInfoUnderConstruction convertToBlockUnderConstruction(
BlockUCState s, DatanodeStorageInfo[] targets) {
if(isComplete()) {
- BlockInfoContiguousUnderConstruction ucBlock =
- new BlockInfoContiguousUnderConstruction(this,
- getBlockCollection().getPreferredBlockReplication(), s, targets);
- ucBlock.setBlockCollection(getBlockCollection());
- return ucBlock;
+ return convertCompleteBlockToUC(s, targets);
}
// the block is already under construction
- BlockInfoContiguousUnderConstruction ucBlock =
- (BlockInfoContiguousUnderConstruction)this;
+ BlockInfoUnderConstruction ucBlock =
+ (BlockInfoUnderConstruction)this;
ucBlock.setBlockUCState(s);
ucBlock.setExpectedLocations(targets);
ucBlock.setBlockCollection(getBlockCollection());
return ucBlock;
}
+ /**
+ * Convert a complete block to an under construction block.
+ */
+ abstract BlockInfoUnderConstruction convertCompleteBlockToUC(
+ BlockUCState s, DatanodeStorageInfo[] targets);
+
@Override
public int hashCode() {
// Super implementation is sufficient
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index eff89a80832..b9abcd03f29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -19,13 +19,13 @@ 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.server.common.HdfsServerConstants;
/**
* Subclass of {@link BlockInfo}, used for a block with replication scheme.
*/
@InterfaceAudience.Private
public class BlockInfoContiguous extends BlockInfo {
- public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
public BlockInfoContiguous(short size) {
super(size);
@@ -40,84 +40,37 @@ public class BlockInfoContiguous extends BlockInfo {
* This is used to convert BlockReplicationInfoUnderConstruction
* @param from BlockReplicationInfo to copy from.
*/
- protected BlockInfoContiguous(BlockInfoContiguous from) {
+ protected BlockInfoContiguous(BlockInfo from) {
super(from);
}
- /**
- * Ensure that there is enough space to include num more triplets.
- * @return first free triplet index.
- */
- private int ensureCapacity(int num) {
- assert this.triplets != null : "BlockInfo is not initialized";
- int last = numNodes();
- if (triplets.length >= (last+num)*3) {
- return last;
- }
- /* Not enough space left. Create a new array. Should normally
- * happen only when replication is manually increased by the user. */
- Object[] old = triplets;
- triplets = new Object[(last+num)*3];
- System.arraycopy(old, 0, triplets, 0, last * 3);
- return last;
- }
-
@Override
boolean addStorage(DatanodeStorageInfo storage) {
- // find the last null node
- int lastNode = ensureCapacity(1);
- setStorageInfo(lastNode, storage);
- setNext(lastNode, null);
- setPrevious(lastNode, null);
- return true;
+ return ContiguousBlockStorageOp.addStorage(this, storage);
}
@Override
boolean removeStorage(DatanodeStorageInfo storage) {
- int dnIndex = findStorageInfo(storage);
- if (dnIndex < 0) { // the node is not found
- return false;
- }
- assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
- "Block is still in the list and must be removed first.";
- // find the last not null node
- int lastNode = numNodes()-1;
- // replace current node triplet by the lastNode one
- setStorageInfo(dnIndex, getStorageInfo(lastNode));
- setNext(dnIndex, getNext(lastNode));
- setPrevious(dnIndex, getPrevious(lastNode));
- // set the last triplet to null
- setStorageInfo(lastNode, null);
- setNext(lastNode, null);
- setPrevious(lastNode, null);
- return true;
+ return ContiguousBlockStorageOp.removeStorage(this, storage);
}
@Override
public int numNodes() {
- assert this.triplets != null : "BlockInfo is not initialized";
- assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-
- for (int idx = getCapacity()-1; idx >= 0; idx--) {
- if (getDatanode(idx) != null) {
- return idx + 1;
- }
- }
- return 0;
+ return ContiguousBlockStorageOp.numNodes(this);
}
@Override
void replaceBlock(BlockInfo newBlock) {
- assert newBlock instanceof BlockInfoContiguous;
- for (int i = this.numNodes() - 1; i >= 0; i--) {
- final DatanodeStorageInfo storage = this.getStorageInfo(i);
- final boolean removed = storage.removeBlock(this);
- assert removed : "currentBlock not found.";
+ ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+ }
- final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
- newBlock);
- assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
- "newBlock already exists.";
- }
+ @Override
+ BlockInfoUnderConstruction convertCompleteBlockToUC(
+ HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+ BlockInfoUnderConstructionContiguous ucBlock =
+ new BlockInfoUnderConstructionContiguous(this,
+ getBlockCollection().getPreferredBlockReplication(), s, targets);
+ ucBlock.setBlockCollection(getBlockCollection());
+ return ucBlock;
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
similarity index 83%
rename from hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
rename to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
index b7571716393..9cd3987a5cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -22,7 +22,9 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
+import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
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;
@@ -31,15 +33,15 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
* Represents a block that is currently being constructed.
* This is usually the last block of a file opened for write or append.
*/
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
+public abstract class BlockInfoUnderConstruction extends BlockInfo {
/** Block state. See {@link BlockUCState} */
- private BlockUCState blockUCState;
+ protected BlockUCState blockUCState;
/**
* Block replicas as assigned when the block was allocated.
* This defines the pipeline order.
*/
- private List replicas;
+ protected List replicas;
/**
* Index of the primary data node doing the recovery. Useful for log
@@ -57,12 +59,12 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
/**
* The block source to use in the event of copy-on-write truncate.
*/
- private Block truncateBlock;
+ protected Block truncateBlock;
/**
* ReplicaUnderConstruction contains information about replicas while
* they are under construction.
- * The GS, the length and the state of the replica is as reported by
+ * The GS, the length and the state of the replica is as reported by
* the data-node.
* It is not guaranteed, but expected, that data-nodes actually have
* corresponding replicas.
@@ -143,7 +145,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
appendStringTo(b);
return b.toString();
}
-
+
@Override
public void appendStringTo(StringBuilder sb) {
sb.append("ReplicaUC[")
@@ -158,45 +160,24 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
* Create block and set its state to
* {@link BlockUCState#UNDER_CONSTRUCTION}.
*/
- public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
+ public BlockInfoUnderConstruction(Block blk, short replication) {
this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
}
/**
* Create a block that is currently being constructed.
*/
- public BlockInfoContiguousUnderConstruction(Block blk, short replication,
+ public BlockInfoUnderConstruction(Block blk, short replication,
BlockUCState state, DatanodeStorageInfo[] targets) {
super(blk, replication);
- assert getBlockUCState() != BlockUCState.COMPLETE :
- "BlockInfoUnderConstruction cannot be in COMPLETE state";
+ Preconditions.checkState(getBlockUCState() != BlockUCState.COMPLETE,
+ "BlockInfoUnderConstruction cannot be in COMPLETE state");
this.blockUCState = state;
setExpectedLocations(targets);
}
- /**
- * Convert an under construction block to a complete block.
- *
- * @return BlockInfo - a complete block.
- * @throws IOException if the state of the block
- * (the generation stamp and the length) has not been committed by
- * the client or it does not have at least a minimal number of replicas
- * reported from data-nodes.
- */
- BlockInfo convertToCompleteBlock() throws IOException {
- assert getBlockUCState() != BlockUCState.COMPLETE :
- "Trying to convert a COMPLETE block";
- return new BlockInfoContiguous(this);
- }
-
- /** Set expected locations */
- public void setExpectedLocations(DatanodeStorageInfo[] targets) {
- int numLocations = targets == null ? 0 : targets.length;
- this.replicas = new ArrayList(numLocations);
- for(int i = 0; i < numLocations; i++)
- replicas.add(
- new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
- }
+ /** Set expected locations. */
+ public abstract void setExpectedLocations(DatanodeStorageInfo[] targets);
/**
* Create array of expected replica locations
@@ -205,12 +186,13 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
public DatanodeStorageInfo[] getExpectedStorageLocations() {
int numLocations = replicas == null ? 0 : replicas.size();
DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
- for(int i = 0; i < numLocations; i++)
+ for(int i = 0; i < numLocations; i++) {
storages[i] = replicas.get(i).getExpectedStorageLocation();
+ }
return storages;
}
- /** Get the number of expected locations */
+ /** Get the number of expected locations. */
public int getNumExpectedLocations() {
return replicas == null ? 0 : replicas.size();
}
@@ -228,19 +210,15 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
blockUCState = s;
}
- /** Get block recovery ID */
+ /** Get block recovery ID. */
public long getBlockRecoveryId() {
return blockRecoveryId;
}
- /** Get recover block */
- public Block getTruncateBlock() {
- return truncateBlock;
- }
+ /** Get recover block. */
+ public abstract Block getTruncateBlock();
- public void setTruncateBlock(Block recoveryBlock) {
- this.truncateBlock = recoveryBlock;
- }
+ public abstract void setTruncateBlock(Block recoveryBlock);
/**
* Process the recorded replicas. When about to commit or finish the
@@ -250,8 +228,9 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
public void setGenerationStampAndVerifyReplicas(long genStamp) {
// Set the generation stamp for the block.
setGenerationStamp(genStamp);
- if (replicas == null)
+ if (replicas == null) {
return;
+ }
// Remove the replicas with wrong gen stamp.
// The replica list is unchanged.
@@ -267,13 +246,14 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
/**
* Commit block's length and generation stamp as reported by the client.
* Set block state to {@link BlockUCState#COMMITTED}.
- * @param block - contains client reported block length and generation
+ * @param block - contains client reported block length and generation
* @throws IOException if block ids are inconsistent.
*/
void commitBlock(Block block) throws IOException {
- if(getBlockId() != block.getBlockId())
+ if(getBlockId() != block.getBlockId()) {
throw new IOException("Trying to commit inconsistent block: id = "
+ block.getBlockId() + ", expected id = " + getBlockId());
+ }
blockUCState = BlockUCState.COMMITTED;
this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
// Sort out invalid replicas.
@@ -289,16 +269,17 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
setBlockUCState(BlockUCState.UNDER_RECOVERY);
blockRecoveryId = recoveryId;
if (replicas.size() == 0) {
- NameNode.blockStateChangeLog.warn("BLOCK*"
- + " BlockInfoUnderConstruction.initLeaseRecovery:"
- + " No blocks found, lease removed.");
+ NameNode.blockStateChangeLog.warn("BLOCK* " +
+ "BlockInfoUnderConstruction.initLeaseRecovery: " +
+ "No blocks found, lease removed.");
}
boolean allLiveReplicasTriedAsPrimary = true;
for (int i = 0; i < replicas.size(); i++) {
// Check if all replicas have been tried or not.
if (replicas.get(i).isAlive()) {
allLiveReplicasTriedAsPrimary =
- (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
+ (allLiveReplicasTriedAsPrimary &&
+ replicas.get(i).getChosenAsPrimary());
}
}
if (allLiveReplicasTriedAsPrimary) {
@@ -312,7 +293,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
primaryNodeIndex = -1;
for(int i = 0; i < replicas.size(); i++) {
// Skip alive replicas which have been chosen for recovery.
- if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
+ if (!(replicas.get(i).isAlive() &&
+ !replicas.get(i).getChosenAsPrimary())) {
continue;
}
final ReplicaUnderConstruction ruc = replicas.get(i);
@@ -325,7 +307,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
}
}
if (primary != null) {
- primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
+ primary.getExpectedStorageLocation().
+ getDatanodeDescriptor().addBlockToBeRecovered(this);
primary.setChosenAsPrimary(true);
NameNode.blockStateChangeLog.info(
"BLOCK* {} recovery started, primary={}", this, primary);
@@ -358,6 +341,25 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
replicas.add(new ReplicaUnderConstruction(block, storage, rState));
}
+ /**
+ * Convert an under construction block to a complete block.
+ *
+ * @return a complete block.
+ * @throws IOException
+ * if the state of the block (the generation stamp and the length)
+ * has not been committed by the client or it does not have at
+ * least a minimal number of replicas reported from data-nodes.
+ */
+ public abstract BlockInfo convertToCompleteBlock();
+
+ @Override
+ BlockInfoUnderConstruction convertCompleteBlockToUC
+ (HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+ BlockManager.LOG.error("convertCompleteBlockToUC should only be applied " +
+ "on complete blocks.");
+ return null;
+ }
+
@Override // BlockInfo
// BlockInfoUnderConstruction participates in maps the same way as BlockInfo
public int hashCode() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
new file mode 100644
index 00000000000..c66675a29a4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
@@ -0,0 +1,110 @@
+/**
+ * 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.server.blockmanagement;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import java.util.ArrayList;
+
+/**
+ * Subclass of {@link BlockInfoUnderConstruction}, representing a block under
+ * the contiguous (instead of striped) layout.
+ */
+public class BlockInfoUnderConstructionContiguous extends
+ BlockInfoUnderConstruction {
+ /**
+ * Create block and set its state to
+ * {@link HdfsServerConstants.BlockUCState#UNDER_CONSTRUCTION}.
+ */
+ public BlockInfoUnderConstructionContiguous(Block blk, short replication) {
+ this(blk, replication, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+ null);
+ }
+
+ /**
+ * Create a block that is currently being constructed.
+ */
+ public BlockInfoUnderConstructionContiguous(Block blk, short replication,
+ HdfsServerConstants.BlockUCState state, DatanodeStorageInfo[] targets) {
+ super(blk, replication);
+ Preconditions.checkState(getBlockUCState() !=
+ HdfsServerConstants.BlockUCState.COMPLETE,
+ "BlockInfoUnderConstructionContiguous cannot be in COMPLETE state");
+ this.blockUCState = state;
+ setExpectedLocations(targets);
+ }
+
+ /**
+ * Convert an under construction block to a complete block.
+ *
+ * @return BlockInfo - a complete block.
+ * @throws IOException if the state of the block
+ * (the generation stamp and the length) has not been committed by
+ * the client or it does not have at least a minimal number of replicas
+ * reported from data-nodes.
+ */
+ @Override
+ public BlockInfoContiguous convertToCompleteBlock() {
+ Preconditions.checkState(getBlockUCState() !=
+ HdfsServerConstants.BlockUCState.COMPLETE,
+ "Trying to convert a COMPLETE block");
+ return new BlockInfoContiguous(this);
+ }
+
+ @Override
+ boolean addStorage(DatanodeStorageInfo storage) {
+ return ContiguousBlockStorageOp.addStorage(this, storage);
+ }
+
+ @Override
+ boolean removeStorage(DatanodeStorageInfo storage) {
+ return ContiguousBlockStorageOp.removeStorage(this, storage);
+ }
+
+ @Override
+ public int numNodes() {
+ return ContiguousBlockStorageOp.numNodes(this);
+ }
+
+ @Override
+ void replaceBlock(BlockInfo newBlock) {
+ ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+ }
+
+ @Override
+ public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+ int numLocations = targets == null ? 0 : targets.length;
+ this.replicas = new ArrayList<>(numLocations);
+ for(int i = 0; i < numLocations; i++) {
+ replicas.add(
+ new ReplicaUnderConstruction(this, targets[i], HdfsServerConstants.ReplicaState.RBW));
+ }
+ }
+
+ @Override
+ public Block getTruncateBlock() {
+ return truncateBlock;
+ }
+
+ @Override
+ public void setTruncateBlock(Block recoveryBlock) {
+ this.truncateBlock = recoveryBlock;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4562d94cbe5..ebc9017a111 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -602,7 +602,7 @@ public class BlockManager {
* of replicas reported from data-nodes.
*/
private static boolean commitBlock(
- final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
+ final BlockInfoUnderConstruction block, final Block commitBlock)
throws IOException {
if (block.getBlockUCState() == BlockUCState.COMMITTED)
return false;
@@ -634,7 +634,7 @@ public class BlockManager {
return false; // already completed (e.g. by syncBlock)
final boolean b = commitBlock(
- (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
+ (BlockInfoUnderConstruction) lastBlock, commitBlock);
if(countNodes(lastBlock).liveReplicas() >= minReplication)
completeBlock(bc, bc.numBlocks()-1, false);
return b;
@@ -654,8 +654,8 @@ public class BlockManager {
BlockInfo curBlock = bc.getBlocks()[blkIndex];
if(curBlock.isComplete())
return curBlock;
- BlockInfoContiguousUnderConstruction ucBlock =
- (BlockInfoContiguousUnderConstruction) curBlock;
+ BlockInfoUnderConstruction ucBlock =
+ (BlockInfoUnderConstruction) curBlock;
int numNodes = ucBlock.numNodes();
if (!force && numNodes < minReplication)
throw new IOException("Cannot complete block: " +
@@ -697,7 +697,7 @@ public class BlockManager {
* when tailing edit logs as a Standby.
*/
public BlockInfo forceCompleteBlock(final BlockCollection bc,
- final BlockInfoContiguousUnderConstruction block) throws IOException {
+ final BlockInfoUnderConstruction block) throws IOException {
block.commitBlock(block);
return completeBlock(bc, block, true);
}
@@ -728,7 +728,7 @@ public class BlockManager {
DatanodeStorageInfo[] targets = getStorages(oldBlock);
- BlockInfoContiguousUnderConstruction ucBlock =
+ BlockInfoUnderConstruction ucBlock =
bc.setLastBlock(oldBlock, targets);
blocksMap.replaceBlock(ucBlock);
@@ -830,14 +830,14 @@ public class BlockManager {
/** @return a LocatedBlock for the given block */
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
) throws IOException {
- if (blk instanceof BlockInfoContiguousUnderConstruction) {
+ if (blk instanceof BlockInfoUnderConstruction) {
if (blk.isComplete()) {
throw new IOException(
"blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
+ ", blk=" + blk);
}
- final BlockInfoContiguousUnderConstruction uc =
- (BlockInfoContiguousUnderConstruction) blk;
+ final BlockInfoUnderConstruction uc =
+ (BlockInfoUnderConstruction) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
return newLocatedBlock(eb, storages, pos, false);
@@ -1744,11 +1744,11 @@ public class BlockManager {
* reported by the datanode in the block report.
*/
static class StatefulBlockInfo {
- final BlockInfoContiguousUnderConstruction storedBlock;
+ final BlockInfoUnderConstruction storedBlock;
final Block reportedBlock;
final ReplicaState reportedState;
- StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
+ StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
Block reportedBlock, ReplicaState reportedState) {
this.storedBlock = storedBlock;
this.reportedBlock = reportedBlock;
@@ -1789,7 +1789,7 @@ public class BlockManager {
BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
Reason reasonCode) {
- this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
+ this(new BlockInfoContiguous(stored), stored,
reason, reasonCode);
//the corrupted block in datanode has a different generation stamp
corrupted.setGenerationStamp(gs);
@@ -2148,13 +2148,13 @@ public class BlockManager {
// If block is under construction, add this replica to its list
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
- ((BlockInfoContiguousUnderConstruction)storedBlock)
+ ((BlockInfoUnderConstruction)storedBlock)
.addReplicaIfNotPresent(storageInfo, iblk, reportedState);
// OpenFileBlocks only inside snapshots also will be added to safemode
// threshold. So we need to update such blocks to safemode
// refer HDFS-5283
- BlockInfoContiguousUnderConstruction blockUC =
- (BlockInfoContiguousUnderConstruction) storedBlock;
+ BlockInfoUnderConstruction blockUC =
+ (BlockInfoUnderConstruction) storedBlock;
if (namesystem.isInSnapshot(blockUC)) {
int numOfReplicas = blockUC.getNumExpectedLocations();
namesystem.incrementSafeBlockCount(numOfReplicas);
@@ -2309,7 +2309,7 @@ public class BlockManager {
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
toUC.add(new StatefulBlockInfo(
- (BlockInfoContiguousUnderConstruction) storedBlock,
+ (BlockInfoUnderConstruction) storedBlock,
new Block(block), reportedState));
return storedBlock;
}
@@ -2500,7 +2500,7 @@ public class BlockManager {
void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
DatanodeStorageInfo storageInfo) throws IOException {
- BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
+ BlockInfoUnderConstruction block = ucBlock.storedBlock;
block.addReplicaIfNotPresent(
storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
@@ -2561,7 +2561,7 @@ public class BlockManager {
assert block != null && namesystem.hasWriteLock();
BlockInfo storedBlock;
DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
- if (block instanceof BlockInfoContiguousUnderConstruction) {
+ if (block instanceof BlockInfoUnderConstruction) {
//refresh our copy in case the block got completed in another thread
storedBlock = blocksMap.getStoredBlock(block);
} else {
@@ -3499,8 +3499,8 @@ public class BlockManager {
String src, BlockInfo[] blocks) {
for (BlockInfo b: blocks) {
if (!b.isComplete()) {
- final BlockInfoContiguousUnderConstruction uc =
- (BlockInfoContiguousUnderConstruction)b;
+ final BlockInfoUnderConstruction uc =
+ (BlockInfoUnderConstruction)b;
final int numNodes = b.numNodes();
LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
+ uc.getBlockUCState() + ", replication# = " + numNodes
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
new file mode 100644
index 00000000000..092f65ec3cb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
@@ -0,0 +1,106 @@
+/**
+ * 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.server.blockmanagement;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility class with logic on managing storage locations shared between
+ * complete and under-construction blocks under the contiguous format --
+ * {@link BlockInfoContiguous} and
+ * {@link BlockInfoUnderConstructionContiguous}.
+ */
+class ContiguousBlockStorageOp {
+ /**
+ * Ensure that there is enough space to include num more triplets.
+ * @return first free triplet index.
+ */
+ private static int ensureCapacity(BlockInfo b, int num) {
+ Preconditions.checkArgument(b.triplets != null,
+ "BlockInfo is not initialized");
+ int last = b.numNodes();
+ if (b.triplets.length >= (last+num)*3) {
+ return last;
+ }
+ /* Not enough space left. Create a new array. Should normally
+ * happen only when replication is manually increased by the user. */
+ Object[] old = b.triplets;
+ b.triplets = new Object[(last+num)*3];
+ System.arraycopy(old, 0, b.triplets, 0, last * 3);
+ return last;
+ }
+
+ static boolean addStorage(BlockInfo b, DatanodeStorageInfo storage) {
+ // find the last null node
+ int lastNode = ensureCapacity(b, 1);
+ b.setStorageInfo(lastNode, storage);
+ b.setNext(lastNode, null);
+ b.setPrevious(lastNode, null);
+ return true;
+ }
+
+ static boolean removeStorage(BlockInfo b,
+ DatanodeStorageInfo storage) {
+ int dnIndex = b.findStorageInfo(storage);
+ if (dnIndex < 0) { // the node is not found
+ return false;
+ }
+ Preconditions.checkArgument(b.getPrevious(dnIndex) == null &&
+ b.getNext(dnIndex) == null,
+ "Block is still in the list and must be removed first.");
+ // find the last not null node
+ int lastNode = b.numNodes()-1;
+ // replace current node triplet by the lastNode one
+ b.setStorageInfo(dnIndex, b.getStorageInfo(lastNode));
+ b.setNext(dnIndex, b.getNext(lastNode));
+ b.setPrevious(dnIndex, b.getPrevious(lastNode));
+ // set the last triplet to null
+ b.setStorageInfo(lastNode, null);
+ b.setNext(lastNode, null);
+ b.setPrevious(lastNode, null);
+ return true;
+ }
+
+ static int numNodes(BlockInfo b) {
+ Preconditions.checkArgument(b.triplets != null,
+ "BlockInfo is not initialized");
+ Preconditions.checkArgument(b.triplets.length % 3 == 0,
+ "Malformed BlockInfo");
+
+ for (int idx = b.getCapacity()-1; idx >= 0; idx--) {
+ if (b.getDatanode(idx) != null) {
+ return idx + 1;
+ }
+ }
+ return 0;
+ }
+
+ static void replaceBlock(BlockInfo b, BlockInfo newBlock) {
+ for (int i = b.numNodes() - 1; i >= 0; i--) {
+ final DatanodeStorageInfo storage = b.getStorageInfo(i);
+ final boolean removed = storage.removeBlock(b);
+ Preconditions.checkState(removed, "currentBlock not found.");
+
+ final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+ newBlock);
+ Preconditions.checkState(
+ result == DatanodeStorageInfo.AddBlockResult.ADDED,
+ "newBlock already exists.");
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 02abc1dc9eb..dd7b3011df4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -219,8 +219,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
/** A queue of blocks to be replicated by this datanode */
private final BlockQueue replicateBlocks = new BlockQueue();
/** A queue of blocks to be recovered by this datanode */
- private final BlockQueue recoverBlocks =
- new BlockQueue();
+ private final BlockQueue recoverBlocks =
+ new BlockQueue();
/** A set of blocks to be invalidated by this datanode */
private final LightWeightHashSet invalidateBlocks = new LightWeightHashSet();
@@ -599,7 +599,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Store block recovery work.
*/
- void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+ void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
if(recoverBlocks.contains(block)) {
// this prevents adding the same block twice to the recovery queue
BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -641,11 +641,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
return replicateBlocks.poll(maxTransfers);
}
- public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
- List blocks = recoverBlocks.poll(maxTransfers);
+ public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(
+ int maxTransfers) {
+ List blocks = recoverBlocks.poll(maxTransfers);
if(blocks == null)
return null;
- return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+ return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 58349cced7b..8143fb44a11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1380,12 +1380,12 @@ public class DatanodeManager {
}
//check lease recovery
- BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+ BlockInfoUnderConstruction[] blocks = nodeinfo
.getLeaseRecoveryCommand(Integer.MAX_VALUE);
if (blocks != null) {
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
blocks.length);
- for (BlockInfoContiguousUnderConstruction b : blocks) {
+ for (BlockInfoUnderConstruction b : blocks) {
final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
// Skip stale nodes during recovery - not heart beated for some time (30s by default).
final List recoveryLocations =
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 3b5f973df8f..4830d5d864a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -43,7 +43,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -73,7 +74,7 @@ class FSDirWriteFileOp {
Block block) throws IOException {
// modify file-> block and blocksMap
// fileNode should be under construction
- BlockInfoContiguousUnderConstruction uc = fileNode.removeLastBlock(block);
+ BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
if (uc == null) {
return false;
}
@@ -236,7 +237,7 @@ class FSDirWriteFileOp {
} else {
// add new chosen targets to already allocated block and return
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
- ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
+ ((BlockInfoUnderConstruction) lastBlockInFile)
.setExpectedLocations(targets);
offset = pendingFile.computeFileSize();
return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
@@ -520,8 +521,8 @@ class FSDirWriteFileOp {
fileINode.getPreferredBlockReplication(), true);
// associate new last block for the file
- BlockInfoContiguousUnderConstruction blockInfo =
- new BlockInfoContiguousUnderConstruction(
+ BlockInfoUnderConstruction blockInfo =
+ new BlockInfoUnderConstructionContiguous(
block,
fileINode.getFileReplication(),
HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
@@ -662,8 +663,8 @@ class FSDirWriteFileOp {
"allocation of a new block in " + src + ". Returning previously" +
" allocated block " + lastBlockInFile);
long offset = file.computeFileSize();
- BlockInfoContiguousUnderConstruction lastBlockUC =
- (BlockInfoContiguousUnderConstruction) lastBlockInFile;
+ BlockInfoUnderConstruction lastBlockUC =
+ (BlockInfoUnderConstruction) lastBlockInFile;
onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
lastBlockUC.getExpectedStorageLocations(), offset);
return new FileState(file, src, iip);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 3de676080f3..df01edd74b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -45,7 +45,8 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -960,16 +961,16 @@ public class FSEditLogLoader {
}
oldLastBlock.setNumBytes(pBlock.getNumBytes());
- if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
+ if (oldLastBlock instanceof BlockInfoUnderConstruction) {
fsNamesys.getBlockManager().forceCompleteBlock(file,
- (BlockInfoContiguousUnderConstruction) oldLastBlock);
+ (BlockInfoUnderConstruction) oldLastBlock);
fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
}
} else { // the penultimate block is null
Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
}
// add the new block
- BlockInfo newBI = new BlockInfoContiguousUnderConstruction(
+ BlockInfo newBI = new BlockInfoUnderConstructionContiguous(
newBlock, file.getPreferredBlockReplication());
fsNamesys.getBlockManager().addBlockCollection(newBI, file);
file.addBlock(newBI);
@@ -1010,11 +1011,11 @@ public class FSEditLogLoader {
oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
- if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+ if (oldBlock instanceof BlockInfoUnderConstruction &&
(!isLastBlock || op.shouldCompleteLastBlock())) {
changeMade = true;
fsNamesys.getBlockManager().forceCompleteBlock(file,
- (BlockInfoContiguousUnderConstruction) oldBlock);
+ (BlockInfoUnderConstruction) oldBlock);
}
if (changeMade) {
// The state or gen-stamp of the block has changed. So, we may be
@@ -1049,7 +1050,7 @@ public class FSEditLogLoader {
// TODO: shouldn't this only be true for the last block?
// what about an old-version fsync() where fsync isn't called
// until several blocks in?
- newBI = new BlockInfoContiguousUnderConstruction(
+ newBI = new BlockInfoUnderConstructionContiguous(
newBlock, file.getPreferredBlockReplication());
} else {
// OP_CLOSE should add finalized blocks. This code path
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 30517d06de7..2305e31d909 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -777,8 +777,9 @@ public class FSImageFormat {
// convert the last block to BlockUC
if (blocks.length > 0) {
BlockInfo lastBlk = blocks[blocks.length - 1];
- blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
- lastBlk, replication);
+ blocks[blocks.length - 1] =
+ new BlockInfoUnderConstructionContiguous(
+ lastBlk, replication);
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index e8378e58f77..d90751cf176 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -364,8 +364,8 @@ public final class FSImageFormatPBINode {
if (blocks.length > 0) {
BlockInfo lastBlk = file.getLastBlock();
// replace the last block of file
- file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
- lastBlk, replication));
+ file.setBlock(file.numBlocks() - 1,
+ new BlockInfoUnderConstructionContiguous(lastBlk, replication));
}
}
return file;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index f71cf0b765c..2dc6252a0db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -137,7 +137,7 @@ public class FSImageSerialization {
// last block is UNDER_CONSTRUCTION
if(numBlocks > 0) {
blk.readFields(in);
- blocks[i] = new BlockInfoContiguousUnderConstruction(
+ blocks[i] = new BlockInfoUnderConstructionContiguous(
blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
}
PermissionStatus perm = PermissionStatus.read(in);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d3b32dad6cc..ef53692c444 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -204,7 +204,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2003,7 +2004,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final BlockInfo last = file.getLastBlock();
if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
final Block truncateBlock
- = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
+ = ((BlockInfoUnderConstruction)last).getTruncateBlock();
if (truncateBlock != null) {
final long truncateLength = file.computeFileSize(false, false)
+ truncateBlock.getNumBytes();
@@ -2082,11 +2083,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
}
- BlockInfoContiguousUnderConstruction truncatedBlockUC;
+ BlockInfoUnderConstruction truncatedBlockUC;
if(shouldCopyOnTruncate) {
// Add new truncateBlock into blocksMap and
// use oldBlock as a source for copy-on-truncate recovery
- truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+ truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
file.getPreferredBlockReplication());
truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
truncatedBlockUC.setTruncateBlock(oldBlock);
@@ -2102,7 +2103,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
oldBlock = file.getLastBlock();
assert !oldBlock.isComplete() : "oldBlock should be under construction";
- truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
+ truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
truncatedBlockUC.getTruncateBlock().setNumBytes(
oldBlock.getNumBytes() - lastBlockDelta);
@@ -3519,7 +3520,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
throw new AlreadyBeingCreatedException(message);
case UNDER_CONSTRUCTION:
case UNDER_RECOVERY:
- final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
+ final BlockInfoUnderConstruction uc =
+ (BlockInfoUnderConstruction)lastBlock;
// determine if last block was intended to be truncated
Block recoveryBlock = uc.getTruncateBlock();
boolean truncateRecovery = recoveryBlock != null;
@@ -3635,7 +3637,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
@Override
- public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
+ public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
assert hasReadLock();
final BlockCollection bc = blockUC.getBlockCollection();
if (bc == null || !(bc instanceof INodeFile)
@@ -3682,7 +3684,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
waitForLoadingFSImage();
writeLock();
boolean copyTruncate = false;
- BlockInfoContiguousUnderConstruction truncatedBlock = null;
+ BlockInfoUnderConstruction truncatedBlock = null;
try {
checkOperation(OperationCategory.WRITE);
// If a DN tries to commit to the standby, the recovery will
@@ -3739,7 +3741,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return;
}
- truncatedBlock = (BlockInfoContiguousUnderConstruction) iFile
+ truncatedBlock = (BlockInfoUnderConstruction) iFile
.getLastBlock();
long recoveryId = truncatedBlock.getBlockRecoveryId();
copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
@@ -5774,8 +5776,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
assert hasWriteLock();
// check the vadility of the block and lease holder name
final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
- final BlockInfoContiguousUnderConstruction blockinfo
- = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
+ final BlockInfoUnderConstruction blockinfo
+ = (BlockInfoUnderConstruction)pendingFile.getLastBlock();
// check new GS & length: this is not expected
if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
index d07ae1f513b..74c5d094991 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
@@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
/**
@@ -61,7 +61,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
BlockInfo lastBlock = f.getLastBlock();
assert (lastBlock != null) : "The last block for path "
+ f.getFullPathName() + " is null when updating its length";
- assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
+ assert (lastBlock instanceof BlockInfoUnderConstruction)
: "The last block for path " + f.getFullPathName()
+ " is not a BlockInfoUnderConstruction when updating its length";
lastBlock.setNumBytes(lastBlockLength);
@@ -76,9 +76,9 @@ public class FileUnderConstructionFeature implements INode.Feature {
final BlocksMapUpdateInfo collectedBlocks) {
final BlockInfo[] blocks = f.getBlocks();
if (blocks != null && blocks.length > 0
- && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
- BlockInfoContiguousUnderConstruction lastUC =
- (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
+ && blocks[blocks.length - 1] instanceof BlockInfoUnderConstruction) {
+ BlockInfoUnderConstruction lastUC =
+ (BlockInfoUnderConstruction) blocks[blocks.length - 1];
if (lastUC.getNumBytes() == 0) {
// this is a 0-sized block. do not need check its UC state here
collectedBlocks.addDeleteBlock(lastUC);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 294323ca34b..4590eecb736 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -231,7 +231,7 @@ public class INodeFile extends INodeWithAdditionalFields
}
@Override // BlockCollection, the file should be under construction
- public BlockInfoContiguousUnderConstruction setLastBlock(
+ public BlockInfoUnderConstruction setLastBlock(
BlockInfo lastBlock, DatanodeStorageInfo[] locations)
throws IOException {
Preconditions.checkState(isUnderConstruction(),
@@ -240,7 +240,7 @@ public class INodeFile extends INodeWithAdditionalFields
if (numBlocks() == 0) {
throw new IOException("Failed to set last block: File is empty.");
}
- BlockInfoContiguousUnderConstruction ucBlock =
+ BlockInfoUnderConstruction ucBlock =
lastBlock.convertToBlockUnderConstruction(
BlockUCState.UNDER_CONSTRUCTION, locations);
setBlock(numBlocks() - 1, ucBlock);
@@ -251,7 +251,7 @@ public class INodeFile extends INodeWithAdditionalFields
* Remove a block from the block list. This block should be
* the last one on the list.
*/
- BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) {
+ BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
Preconditions.checkState(isUnderConstruction(),
"file is no longer under construction");
if (blocks == null || blocks.length == 0) {
@@ -262,8 +262,8 @@ public class INodeFile extends INodeWithAdditionalFields
return null;
}
- BlockInfoContiguousUnderConstruction uc =
- (BlockInfoContiguousUnderConstruction)blocks[size_1];
+ 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);
@@ -689,7 +689,7 @@ public class INodeFile extends INodeWithAdditionalFields
final int last = blocks.length - 1;
//check if the last block is BlockInfoUnderConstruction
long size = blocks[last].getNumBytes();
- if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
+ if (blocks[last] instanceof BlockInfoUnderConstruction) {
if (!includesLastUcBlock) {
size = 0;
} else if (usePreferredBlockSize4LastUcBlock) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 3442e7be6b7..40c4765f91c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.ipc.StandbyException;
@@ -45,5 +45,5 @@ public interface Namesystem extends RwLock, SafeMode {
public void checkOperation(OperationCategory read) throws StandbyException;
- public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+ public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 6b8388e0fe1..d081a6b5dda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -22,7 +22,7 @@ import java.util.List;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@@ -133,7 +133,7 @@ public class FileDiffList extends
Block dontRemoveBlock = null;
if (lastBlock != null && lastBlock.getBlockUCState().equals(
HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
- dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+ dontRemoveBlock = ((BlockInfoUnderConstruction) lastBlock)
.getTruncateBlock();
}
// Collect the remaining blocks of the file, ignoring truncate block
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 50b85c076ef..d06b0248654 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -109,7 +109,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1612,9 +1612,9 @@ public class DFSTestUtil {
BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
assertTrue("Block " + blk + " should be under construction, " +
"got: " + storedBlock,
- storedBlock instanceof BlockInfoContiguousUnderConstruction);
- BlockInfoContiguousUnderConstruction ucBlock =
- (BlockInfoContiguousUnderConstruction)storedBlock;
+ storedBlock instanceof BlockInfoUnderConstruction);
+ BlockInfoUnderConstruction ucBlock =
+ (BlockInfoUnderConstruction)storedBlock;
// We expect that the replica with the most recent heart beat will be
// the one to be in charge of the synchronization / recovery protocol.
final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index a7ba29399dc..630cd1c756e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.util.Time;
import org.junit.Test;
/**
@@ -40,7 +39,8 @@ public class TestBlockInfoUnderConstruction {
DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
- BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+ BlockInfoUnderConstruction blockInfo =
+ new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
(short) 3,
BlockUCState.UNDER_CONSTRUCTION,
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
blockInfo.initializeBlockRecovery(1);
- BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+ BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
assertEquals(blockInfoRecovery[0], blockInfo);
// Recovery attempt #2.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index fd037455a90..5a82b15635f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -726,7 +726,7 @@ public class TestBlockManager {
// verify the storage info is correct
assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
(ds) >= 0);
- assertTrue(((BlockInfoContiguousUnderConstruction) bm.
+ assertTrue(((BlockInfoUnderConstruction) bm.
getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
.findStorageInfo(ds) >= 0);
@@ -747,8 +747,8 @@ public class TestBlockManager {
private BlockInfo addUcBlockToBM(long blkId) {
Block block = new Block(blkId);
- BlockInfoContiguousUnderConstruction blockInfo =
- new BlockInfoContiguousUnderConstruction(block, (short) 3);
+ BlockInfoUnderConstruction blockInfo =
+ new BlockInfoUnderConstructionContiguous(block, (short) 3);
BlockCollection bc = Mockito.mock(BlockCollection.class);
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
bm.blocksMap.addBlockCollection(blockInfo, bc);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
index 6fc30ba6d63..e48e9e84c63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.util.Time;
import org.junit.Test;
/**
@@ -173,7 +172,8 @@ public class TestHeartbeatHandling {
dd1.getStorageInfos()[0],
dd2.getStorageInfos()[0],
dd3.getStorageInfos()[0]};
- BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+ BlockInfoUnderConstruction blockInfo =
+ new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
@@ -195,7 +195,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds.
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
- blockInfo = new BlockInfoContiguousUnderConstruction(
+ blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
@@ -216,7 +216,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds.
DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
- blockInfo = new BlockInfoContiguousUnderConstruction(
+ blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index fccaf3cfc95..6e98538b55d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1176,7 +1176,8 @@ public class TestReplicationPolicy {
// block under construction, the BlockManager will realize the expected
// replication has been achieved and remove it from the under-replicated
// queue.
- BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
+ BlockInfoUnderConstruction info =
+ new BlockInfoUnderConstructionContiguous(block1, (short) 1);
BlockCollection bc = mock(BlockCollection.class);
when(bc.getPreferredBlockReplication()).thenReturn((short)1);
bm.addBlockCollection(info, bc);
@@ -1232,7 +1233,7 @@ public class TestReplicationPolicy {
DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
dataNodes[0], new DatanodeStorage("s1"))};
- final BlockInfoContiguousUnderConstruction ucBlock =
+ final BlockInfoUnderConstruction ucBlock =
info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
storageAry);
DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index f372bec3ba6..872ff9c490f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.junit.AfterClass;
@@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
final List blocks = lb.getLocatedBlocks();
assertEquals(i, blocks.size());
final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
- assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
+ assertTrue(b instanceof BlockInfoUnderConstruction);
if (++i < NUM_BLOCKS) {
// write one more block
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
index bea72412fbf..b6cb5224e7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
@@ -24,7 +24,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.junit.Test;
@@ -68,8 +69,10 @@ public class TestCommitBlockSynchronization {
namesystem.dir.getINodeMap().put(file);
FSNamesystem namesystemSpy = spy(namesystem);
- BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
- block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+ BlockInfoUnderConstruction blockInfo =
+ new BlockInfoUnderConstructionContiguous(
+ block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+ targets);
blockInfo.setBlockCollection(file);
blockInfo.setGenerationStamp(genStamp);
blockInfo.initializeBlockRecovery(genStamp);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 222f22b3fac..df920e0c9bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.security.UserGroupInformation;
@@ -1019,7 +1019,7 @@ public class TestFileTruncate {
is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
- long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+ long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
.getBlockRecoveryId();
assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate(
@@ -1052,7 +1052,7 @@ public class TestFileTruncate {
is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
- long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+ long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
.getBlockRecoveryId();
assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index d202fb788f8..14d9a1ee942 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -752,8 +752,8 @@ public class TestRetryCacheWithHA {
boolean checkNamenodeBeforeReturn() throws Exception {
INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
.getINode4Write(file).asFile();
- BlockInfoContiguousUnderConstruction blkUC =
- (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
+ BlockInfoUnderConstruction blkUC =
+ (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
int datanodeNum = blkUC.getExpectedStorageLocations().length;
for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
Thread.sleep(1000);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index a1abd0892e8..824f45baa41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@ -177,7 +177,7 @@ public class SnapshotTestHelper {
* Specific information for different types of INode:
* {@link INodeDirectory}:childrenSize
* {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
- * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
+ * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
* {@link FileWithSnapshot}: next link
*
* @see INode#dumpTreeRecursively()