|
@@ -17,7 +17,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
@@ -31,7 +30,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
* Represents a block that is currently being constructed.<br>
|
|
|
* This is usually the last block of a file opened for write or append.
|
|
|
*/
|
|
|
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
+public class BlockUnderConstructionFeature {
|
|
|
/** Block state. See {@link BlockUCState} */
|
|
|
private BlockUCState blockUCState;
|
|
|
|
|
@@ -62,32 +61,40 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
/**
|
|
|
* 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.
|
|
|
*/
|
|
|
- static class ReplicaUnderConstruction extends Block {
|
|
|
+ static class ReplicaUnderConstruction {
|
|
|
+ private long generationStamp;
|
|
|
private final DatanodeStorageInfo expectedLocation;
|
|
|
private ReplicaState state;
|
|
|
private boolean chosenAsPrimary;
|
|
|
|
|
|
- ReplicaUnderConstruction(Block block,
|
|
|
- DatanodeStorageInfo target,
|
|
|
- ReplicaState state) {
|
|
|
- super(block);
|
|
|
+ ReplicaUnderConstruction(long generationStamp, DatanodeStorageInfo target,
|
|
|
+ ReplicaState state) {
|
|
|
+ this.generationStamp = generationStamp;
|
|
|
this.expectedLocation = target;
|
|
|
this.state = state;
|
|
|
this.chosenAsPrimary = false;
|
|
|
}
|
|
|
|
|
|
+ long getGenerationStamp() {
|
|
|
+ return this.generationStamp;
|
|
|
+ }
|
|
|
+
|
|
|
+ void setGenerationStamp(long generationStamp) {
|
|
|
+ this.generationStamp = generationStamp;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Expected block replica location as assigned when the block was allocated.
|
|
|
* This defines the pipeline order.
|
|
|
* It is not guaranteed, but expected, that the data-node actually has
|
|
|
* the replica.
|
|
|
*/
|
|
|
- private DatanodeStorageInfo getExpectedStorageLocation() {
|
|
|
+ DatanodeStorageInfo getExpectedStorageLocation() {
|
|
|
return expectedLocation;
|
|
|
}
|
|
|
|
|
@@ -126,76 +133,38 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
return expectedLocation.getDatanodeDescriptor().isAlive;
|
|
|
}
|
|
|
|
|
|
- @Override // Block
|
|
|
- public int hashCode() {
|
|
|
- return super.hashCode();
|
|
|
- }
|
|
|
-
|
|
|
- @Override // Block
|
|
|
- public boolean equals(Object obj) {
|
|
|
- // Sufficient to rely on super's implementation
|
|
|
- return (this == obj) || super.equals(obj);
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
- final StringBuilder b = new StringBuilder(50);
|
|
|
- appendStringTo(b);
|
|
|
+ final StringBuilder b = new StringBuilder(50)
|
|
|
+ .append("ReplicaUC[")
|
|
|
+ .append(expectedLocation)
|
|
|
+ .append("|")
|
|
|
+ .append(state)
|
|
|
+ .append("]");
|
|
|
return b.toString();
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public void appendStringTo(StringBuilder sb) {
|
|
|
- sb.append("ReplicaUC[")
|
|
|
- .append(expectedLocation)
|
|
|
- .append("|")
|
|
|
- .append(state)
|
|
|
- .append("]");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Create block and set its state to
|
|
|
- * {@link BlockUCState#UNDER_CONSTRUCTION}.
|
|
|
- */
|
|
|
- public BlockInfoContiguousUnderConstruction(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,
|
|
|
- BlockUCState state, DatanodeStorageInfo[] targets) {
|
|
|
- super(blk, replication);
|
|
|
+ public BlockUnderConstructionFeature(Block block, BlockUCState state,
|
|
|
+ DatanodeStorageInfo[] targets) {
|
|
|
assert 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);
|
|
|
+ setExpectedLocations(block.getGenerationStamp(), targets);
|
|
|
}
|
|
|
|
|
|
/** Set expected locations */
|
|
|
- public void setExpectedLocations(DatanodeStorageInfo[] targets) {
|
|
|
+ public void setExpectedLocations(long generationStamp,
|
|
|
+ DatanodeStorageInfo[] targets) {
|
|
|
int numLocations = targets == null ? 0 : targets.length;
|
|
|
- this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
|
|
|
- for(int i = 0; i < numLocations; i++)
|
|
|
- replicas.add(
|
|
|
- new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
|
|
|
+ this.replicas = new ArrayList<>(numLocations);
|
|
|
+ for(int i = 0; i < numLocations; i++) {
|
|
|
+ replicas.add(new ReplicaUnderConstruction(generationStamp, targets[i],
|
|
|
+ ReplicaState.RBW));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -205,8 +174,9 @@ 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;
|
|
|
}
|
|
|
|
|
@@ -219,7 +189,6 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
* Return the state of the block under construction.
|
|
|
* @see BlockUCState
|
|
|
*/
|
|
|
- @Override // BlockInfo
|
|
|
public BlockUCState getBlockUCState() {
|
|
|
return blockUCState;
|
|
|
}
|
|
@@ -243,41 +212,23 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Process the recorded replicas. When about to commit or finish the
|
|
|
- * pipeline recovery sort out bad replicas.
|
|
|
- * @param genStamp The final generation stamp for the block.
|
|
|
+ * Set {@link #blockUCState} to {@link BlockUCState#COMMITTED}.
|
|
|
*/
|
|
|
- public void setGenerationStampAndVerifyReplicas(long genStamp) {
|
|
|
- // Set the generation stamp for the block.
|
|
|
- setGenerationStamp(genStamp);
|
|
|
- if (replicas == null)
|
|
|
- return;
|
|
|
-
|
|
|
- // Remove the replicas with wrong gen stamp.
|
|
|
- // The replica list is unchanged.
|
|
|
- for (ReplicaUnderConstruction r : replicas) {
|
|
|
- if (genStamp != r.getGenerationStamp()) {
|
|
|
- r.getExpectedStorageLocation().removeBlock(this);
|
|
|
- NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica "
|
|
|
- + "from location: {}", r.getExpectedStorageLocation());
|
|
|
- }
|
|
|
- }
|
|
|
+ void commit() {
|
|
|
+ blockUCState = BlockUCState.COMMITTED;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * 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
|
|
|
- * @throws IOException if block ids are inconsistent.
|
|
|
- */
|
|
|
- void commitBlock(Block block) throws IOException {
|
|
|
- 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.
|
|
|
- setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
|
|
|
+ List<ReplicaUnderConstruction> getStaleReplicas(long genStamp) {
|
|
|
+ List<ReplicaUnderConstruction> staleReplicas = new ArrayList<>();
|
|
|
+ if (replicas != null) {
|
|
|
+ // Remove replicas with wrong gen stamp. The replica list is unchanged.
|
|
|
+ for (ReplicaUnderConstruction r : replicas) {
|
|
|
+ if (genStamp != r.getGenerationStamp()) {
|
|
|
+ staleReplicas.add(r);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return staleReplicas;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -285,7 +236,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
* Find the first alive data-node starting from the previous primary and
|
|
|
* make it primary.
|
|
|
*/
|
|
|
- public void initializeBlockRecovery(long recoveryId) {
|
|
|
+ public void initializeBlockRecovery(BlockInfo block, long recoveryId) {
|
|
|
setBlockUCState(BlockUCState.UNDER_RECOVERY);
|
|
|
blockRecoveryId = recoveryId;
|
|
|
if (replicas.size() == 0) {
|
|
@@ -294,17 +245,17 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
+ " No blocks found, lease removed.");
|
|
|
}
|
|
|
boolean allLiveReplicasTriedAsPrimary = true;
|
|
|
- for (int i = 0; i < replicas.size(); i++) {
|
|
|
+ for (ReplicaUnderConstruction replica : replicas) {
|
|
|
// Check if all replicas have been tried or not.
|
|
|
- if (replicas.get(i).isAlive()) {
|
|
|
- allLiveReplicasTriedAsPrimary =
|
|
|
- (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
|
|
|
+ if (replica.isAlive()) {
|
|
|
+ allLiveReplicasTriedAsPrimary = allLiveReplicasTriedAsPrimary
|
|
|
+ && replica.getChosenAsPrimary();
|
|
|
}
|
|
|
}
|
|
|
if (allLiveReplicasTriedAsPrimary) {
|
|
|
// Just set all the replicas to be chosen whether they are alive or not.
|
|
|
- for (int i = 0; i < replicas.size(); i++) {
|
|
|
- replicas.get(i).setChosenAsPrimary(false);
|
|
|
+ for (ReplicaUnderConstruction replica : replicas) {
|
|
|
+ replica.setChosenAsPrimary(false);
|
|
|
}
|
|
|
}
|
|
|
long mostRecentLastUpdate = 0;
|
|
@@ -325,28 +276,27 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
}
|
|
|
}
|
|
|
if (primary != null) {
|
|
|
- primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
|
|
|
+ primary.getExpectedStorageLocation().getDatanodeDescriptor()
|
|
|
+ .addBlockToBeRecovered(block);
|
|
|
primary.setChosenAsPrimary(true);
|
|
|
NameNode.blockStateChangeLog.debug(
|
|
|
"BLOCK* {} recovery started, primary={}", this, primary);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void addReplicaIfNotPresent(DatanodeStorageInfo storage,
|
|
|
- Block block,
|
|
|
- ReplicaState rState) {
|
|
|
+ void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block,
|
|
|
+ ReplicaState rState) {
|
|
|
Iterator<ReplicaUnderConstruction> it = replicas.iterator();
|
|
|
while (it.hasNext()) {
|
|
|
ReplicaUnderConstruction r = it.next();
|
|
|
DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
|
|
|
- if(expectedLocation == storage) {
|
|
|
+ if (expectedLocation == storage) {
|
|
|
// Record the gen stamp from the report
|
|
|
r.setGenerationStamp(block.getGenerationStamp());
|
|
|
return;
|
|
|
} else if (expectedLocation != null &&
|
|
|
expectedLocation.getDatanodeDescriptor() ==
|
|
|
storage.getDatanodeDescriptor()) {
|
|
|
-
|
|
|
// The Datanode reported that the block is on a different storage
|
|
|
// than the one chosen by BlockPlacementPolicy. This can occur as
|
|
|
// we allow Datanodes to choose the target storage. Update our
|
|
@@ -355,46 +305,28 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
- replicas.add(new ReplicaUnderConstruction(block, storage, rState));
|
|
|
- }
|
|
|
-
|
|
|
- @Override // BlockInfo
|
|
|
- // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
|
|
|
- public int hashCode() {
|
|
|
- return super.hashCode();
|
|
|
- }
|
|
|
-
|
|
|
- @Override // BlockInfo
|
|
|
- public boolean equals(Object obj) {
|
|
|
- // Sufficient to rely on super's implementation
|
|
|
- return (this == obj) || super.equals(obj);
|
|
|
+ replicas.add(new ReplicaUnderConstruction(block.getGenerationStamp(), storage, rState));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
final StringBuilder b = new StringBuilder(100);
|
|
|
- appendStringTo(b);
|
|
|
+ appendUCParts(b);
|
|
|
return b.toString();
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public void appendStringTo(StringBuilder sb) {
|
|
|
- super.appendStringTo(sb);
|
|
|
- appendUCParts(sb);
|
|
|
- }
|
|
|
-
|
|
|
private void appendUCParts(StringBuilder sb) {
|
|
|
sb.append("{UCState=").append(blockUCState)
|
|
|
- .append(", truncateBlock=" + truncateBlock)
|
|
|
+ .append(", truncateBlock=").append(truncateBlock)
|
|
|
.append(", primaryNodeIndex=").append(primaryNodeIndex)
|
|
|
.append(", replicas=[");
|
|
|
if (replicas != null) {
|
|
|
Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
|
|
|
if (iter.hasNext()) {
|
|
|
- iter.next().appendStringTo(sb);
|
|
|
+ sb.append(iter.next());
|
|
|
while (iter.hasNext()) {
|
|
|
sb.append(", ");
|
|
|
- iter.next().appendStringTo(sb);
|
|
|
+ sb.append(iter.next());
|
|
|
}
|
|
|
}
|
|
|
}
|