|
@@ -17,28 +17,27 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.Iterator;
|
|
|
|
-import java.util.List;
|
|
|
|
-
|
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.List;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
- * Represents a block that is currently being constructed.<br>
|
|
|
|
|
|
+ * Represents the under construction feature of a Block.
|
|
* This is usually the last block of a file opened for write or append.
|
|
* This is usually the last block of a file opened for write or append.
|
|
*/
|
|
*/
|
|
public class BlockUnderConstructionFeature {
|
|
public class BlockUnderConstructionFeature {
|
|
- /** Block state. See {@link BlockUCState} */
|
|
|
|
private BlockUCState blockUCState;
|
|
private BlockUCState blockUCState;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Block replicas as assigned when the block was allocated.
|
|
* Block replicas as assigned when the block was allocated.
|
|
- * This defines the pipeline order.
|
|
|
|
*/
|
|
*/
|
|
- private List<ReplicaUnderConstruction> replicas;
|
|
|
|
|
|
+ private ReplicaUnderConstruction[] replicas;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Index of the primary data node doing the recovery. Useful for log
|
|
* Index of the primary data node doing the recovery. Useful for log
|
|
@@ -58,112 +57,21 @@ public class BlockUnderConstructionFeature {
|
|
*/
|
|
*/
|
|
private Block truncateBlock;
|
|
private 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 data-node.
|
|
|
|
- * It is not guaranteed, but expected, that data-nodes actually have
|
|
|
|
- * corresponding replicas.
|
|
|
|
- */
|
|
|
|
- static class ReplicaUnderConstruction {
|
|
|
|
- private long generationStamp;
|
|
|
|
- private final DatanodeStorageInfo expectedLocation;
|
|
|
|
- private ReplicaState state;
|
|
|
|
- private boolean chosenAsPrimary;
|
|
|
|
-
|
|
|
|
- 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.
|
|
|
|
- */
|
|
|
|
- DatanodeStorageInfo getExpectedStorageLocation() {
|
|
|
|
- return expectedLocation;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Get replica state as reported by the data-node.
|
|
|
|
- */
|
|
|
|
- ReplicaState getState() {
|
|
|
|
- return state;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Whether the replica was chosen for recovery.
|
|
|
|
- */
|
|
|
|
- boolean getChosenAsPrimary() {
|
|
|
|
- return chosenAsPrimary;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Set replica state.
|
|
|
|
- */
|
|
|
|
- void setState(ReplicaState s) {
|
|
|
|
- state = s;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Set whether this replica was chosen for recovery.
|
|
|
|
- */
|
|
|
|
- void setChosenAsPrimary(boolean chosenAsPrimary) {
|
|
|
|
- this.chosenAsPrimary = chosenAsPrimary;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Is data-node the replica belongs to alive.
|
|
|
|
- */
|
|
|
|
- boolean isAlive() {
|
|
|
|
- return expectedLocation.getDatanodeDescriptor().isAlive;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public String toString() {
|
|
|
|
- final StringBuilder b = new StringBuilder(50)
|
|
|
|
- .append("ReplicaUC[")
|
|
|
|
- .append(expectedLocation)
|
|
|
|
- .append("|")
|
|
|
|
- .append(state)
|
|
|
|
- .append("]");
|
|
|
|
- return b.toString();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Create a block that is currently being constructed.
|
|
|
|
- */
|
|
|
|
- public BlockUnderConstructionFeature(Block block, BlockUCState state,
|
|
|
|
- DatanodeStorageInfo[] targets) {
|
|
|
|
- assert getBlockUCState() != BlockUCState.COMPLETE :
|
|
|
|
- "BlockUnderConstructionFeature cannot be in COMPLETE state";
|
|
|
|
|
|
+ public BlockUnderConstructionFeature(Block blk,
|
|
|
|
+ BlockUCState state, DatanodeStorageInfo[] targets) {
|
|
|
|
+ assert getBlockUCState() != COMPLETE :
|
|
|
|
+ "BlockUnderConstructionFeature cannot be in COMPLETE state";
|
|
this.blockUCState = state;
|
|
this.blockUCState = state;
|
|
- setExpectedLocations(block.getGenerationStamp(), targets);
|
|
|
|
|
|
+ setExpectedLocations(blk, targets);
|
|
}
|
|
}
|
|
|
|
|
|
/** Set expected locations */
|
|
/** Set expected locations */
|
|
- public void setExpectedLocations(long generationStamp,
|
|
|
|
- DatanodeStorageInfo[] targets) {
|
|
|
|
|
|
+ public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets) {
|
|
int numLocations = targets == null ? 0 : targets.length;
|
|
int numLocations = targets == null ? 0 : targets.length;
|
|
- this.replicas = new ArrayList<>(numLocations);
|
|
|
|
|
|
+ this.replicas = new ReplicaUnderConstruction[numLocations];
|
|
for(int i = 0; i < numLocations; i++) {
|
|
for(int i = 0; i < numLocations; i++) {
|
|
- replicas.add(new ReplicaUnderConstruction(generationStamp, targets[i],
|
|
|
|
- ReplicaState.RBW));
|
|
|
|
|
|
+ replicas[i] = new ReplicaUnderConstruction(block, targets[i],
|
|
|
|
+ ReplicaState.RBW);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -172,17 +80,17 @@ public class BlockUnderConstructionFeature {
|
|
* (as has been assigned by chooseTargets()).
|
|
* (as has been assigned by chooseTargets()).
|
|
*/
|
|
*/
|
|
public DatanodeStorageInfo[] getExpectedStorageLocations() {
|
|
public DatanodeStorageInfo[] getExpectedStorageLocations() {
|
|
- int numLocations = replicas == null ? 0 : replicas.size();
|
|
|
|
|
|
+ int numLocations = getNumExpectedLocations();
|
|
DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
|
|
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();
|
|
|
|
|
|
+ storages[i] = replicas[i].getExpectedStorageLocation();
|
|
}
|
|
}
|
|
return storages;
|
|
return storages;
|
|
}
|
|
}
|
|
|
|
|
|
/** Get the number of expected locations */
|
|
/** Get the number of expected locations */
|
|
public int getNumExpectedLocations() {
|
|
public int getNumExpectedLocations() {
|
|
- return replicas == null ? 0 : replicas.size();
|
|
|
|
|
|
+ return replicas == null ? 0 : replicas.length;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -197,7 +105,6 @@ public class BlockUnderConstructionFeature {
|
|
blockUCState = s;
|
|
blockUCState = s;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Get block recovery ID */
|
|
|
|
public long getBlockRecoveryId() {
|
|
public long getBlockRecoveryId() {
|
|
return blockRecoveryId;
|
|
return blockRecoveryId;
|
|
}
|
|
}
|
|
@@ -236,13 +143,16 @@ public class BlockUnderConstructionFeature {
|
|
* Find the first alive data-node starting from the previous primary and
|
|
* Find the first alive data-node starting from the previous primary and
|
|
* make it primary.
|
|
* make it primary.
|
|
*/
|
|
*/
|
|
- public void initializeBlockRecovery(BlockInfo block, long recoveryId) {
|
|
|
|
|
|
+ public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
|
|
setBlockUCState(BlockUCState.UNDER_RECOVERY);
|
|
setBlockUCState(BlockUCState.UNDER_RECOVERY);
|
|
blockRecoveryId = recoveryId;
|
|
blockRecoveryId = recoveryId;
|
|
- if (replicas.size() == 0) {
|
|
|
|
- NameNode.blockStateChangeLog.warn("BLOCK*"
|
|
|
|
- + " BlockUnderConstructionFeature.initLeaseRecovery:"
|
|
|
|
- + " No blocks found, lease removed.");
|
|
|
|
|
|
+ if (replicas == null || replicas.length == 0) {
|
|
|
|
+ NameNode.blockStateChangeLog.warn("BLOCK*" +
|
|
|
|
+ " BlockUnderConstructionFeature.initializeBlockRecovery:" +
|
|
|
|
+ " No blocks found, lease removed.");
|
|
|
|
+ // sets primary node index and return.
|
|
|
|
+ primaryNodeIndex = -1;
|
|
|
|
+ return;
|
|
}
|
|
}
|
|
boolean allLiveReplicasTriedAsPrimary = true;
|
|
boolean allLiveReplicasTriedAsPrimary = true;
|
|
for (ReplicaUnderConstruction replica : replicas) {
|
|
for (ReplicaUnderConstruction replica : replicas) {
|
|
@@ -261,12 +171,12 @@ public class BlockUnderConstructionFeature {
|
|
long mostRecentLastUpdate = 0;
|
|
long mostRecentLastUpdate = 0;
|
|
ReplicaUnderConstruction primary = null;
|
|
ReplicaUnderConstruction primary = null;
|
|
primaryNodeIndex = -1;
|
|
primaryNodeIndex = -1;
|
|
- for(int i = 0; i < replicas.size(); i++) {
|
|
|
|
|
|
+ for (int i = 0; i < replicas.length; i++) {
|
|
// Skip alive replicas which have been chosen for recovery.
|
|
// Skip alive replicas which have been chosen for recovery.
|
|
- if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
|
|
|
|
|
|
+ if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
- final ReplicaUnderConstruction ruc = replicas.get(i);
|
|
|
|
|
|
+ final ReplicaUnderConstruction ruc = replicas[i];
|
|
final long lastUpdate = ruc.getExpectedStorageLocation()
|
|
final long lastUpdate = ruc.getExpectedStorageLocation()
|
|
.getDatanodeDescriptor().getLastUpdateMonotonic();
|
|
.getDatanodeDescriptor().getLastUpdateMonotonic();
|
|
if (lastUpdate > mostRecentLastUpdate) {
|
|
if (lastUpdate > mostRecentLastUpdate) {
|
|
@@ -277,35 +187,45 @@ public class BlockUnderConstructionFeature {
|
|
}
|
|
}
|
|
if (primary != null) {
|
|
if (primary != null) {
|
|
primary.getExpectedStorageLocation().getDatanodeDescriptor()
|
|
primary.getExpectedStorageLocation().getDatanodeDescriptor()
|
|
- .addBlockToBeRecovered(block);
|
|
|
|
|
|
+ .addBlockToBeRecovered(blockInfo);
|
|
primary.setChosenAsPrimary(true);
|
|
primary.setChosenAsPrimary(true);
|
|
NameNode.blockStateChangeLog.debug(
|
|
NameNode.blockStateChangeLog.debug(
|
|
"BLOCK* {} recovery started, primary={}", this, primary);
|
|
"BLOCK* {} recovery started, primary={}", this, primary);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- 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) {
|
|
|
|
- // 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
|
|
|
|
- // state by removing the stale entry and adding a new one.
|
|
|
|
- it.remove();
|
|
|
|
- break;
|
|
|
|
|
|
+ /** Add the reported replica if it is not already in the replica list. */
|
|
|
|
+ void addReplicaIfNotPresent(DatanodeStorageInfo storage,
|
|
|
|
+ Block reportedBlock, ReplicaState rState) {
|
|
|
|
+ if (replicas == null) {
|
|
|
|
+ replicas = new ReplicaUnderConstruction[1];
|
|
|
|
+ replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage,
|
|
|
|
+ rState);
|
|
|
|
+ } else {
|
|
|
|
+ for (int i = 0; i < replicas.length; i++) {
|
|
|
|
+ DatanodeStorageInfo expected =
|
|
|
|
+ replicas[i].getExpectedStorageLocation();
|
|
|
|
+ if (expected == storage) {
|
|
|
|
+ replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
|
|
|
|
+ return;
|
|
|
|
+ } else if (expected != null && expected.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
|
|
|
|
+ // state by removing the stale entry and adding a new one.
|
|
|
|
+ replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
|
|
|
|
+ rState);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ ReplicaUnderConstruction[] newReplicas =
|
|
|
|
+ new ReplicaUnderConstruction[replicas.length + 1];
|
|
|
|
+ System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
|
|
|
|
+ newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
|
|
|
|
+ reportedBlock, storage, rState);
|
|
|
|
+ replicas = newReplicas;
|
|
}
|
|
}
|
|
- replicas.add(new ReplicaUnderConstruction(block.getGenerationStamp(), storage, rState));
|
|
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -321,12 +241,11 @@ public class BlockUnderConstructionFeature {
|
|
.append(", primaryNodeIndex=").append(primaryNodeIndex)
|
|
.append(", primaryNodeIndex=").append(primaryNodeIndex)
|
|
.append(", replicas=[");
|
|
.append(", replicas=[");
|
|
if (replicas != null) {
|
|
if (replicas != null) {
|
|
- Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
|
|
|
|
- if (iter.hasNext()) {
|
|
|
|
- sb.append(iter.next());
|
|
|
|
- while (iter.hasNext()) {
|
|
|
|
|
|
+ int i = 0;
|
|
|
|
+ for (ReplicaUnderConstruction r : replicas) {
|
|
|
|
+ r.appendStringTo(sb);
|
|
|
|
+ if (++i < replicas.length) {
|
|
sb.append(", ");
|
|
sb.append(", ");
|
|
- sb.append(iter.next());
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|