|
@@ -19,9 +19,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
+import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
-import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
|
|
|
+
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
+import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
|
|
|
|
|
|
/**
|
|
|
* Keep prioritized queues of under replicated blocks.
|
|
@@ -34,7 +36,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
*
|
|
|
* <p/>
|
|
|
* The policy for choosing which priority to give added blocks
|
|
|
- * is implemented in {@link #getPriority(BlockInfo, int, int, int)}.
|
|
|
+ * is implemented in {@link #getPriority(BlockInfo, int, int, int, int)}.
|
|
|
* </p>
|
|
|
* <p>The queue order is as follows:</p>
|
|
|
* <ol>
|
|
@@ -147,6 +149,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
*/
|
|
|
private int getPriority(BlockInfo block,
|
|
|
int curReplicas,
|
|
|
+ int readOnlyReplicas,
|
|
|
int decommissionedReplicas,
|
|
|
int expectedReplicas) {
|
|
|
assert curReplicas >= 0 : "Negative replicas!";
|
|
@@ -159,19 +162,24 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
return getPriorityStriped(curReplicas, decommissionedReplicas,
|
|
|
sblk.getRealDataBlockNum(), sblk.getParityBlockNum());
|
|
|
} else {
|
|
|
- return getPriorityContiguous(curReplicas, decommissionedReplicas,
|
|
|
- expectedReplicas);
|
|
|
+ return getPriorityContiguous(curReplicas, readOnlyReplicas,
|
|
|
+ decommissionedReplicas, expectedReplicas);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private int getPriorityContiguous(int curReplicas, int decommissionedReplicas,
|
|
|
- int expectedReplicas) {
|
|
|
+ private int getPriorityContiguous(int curReplicas, int readOnlyReplicas,
|
|
|
+ int decommissionedReplicas, int expectedReplicas) {
|
|
|
if (curReplicas == 0) {
|
|
|
// If there are zero non-decommissioned replicas but there are
|
|
|
// some decommissioned replicas, then assign them highest priority
|
|
|
if (decommissionedReplicas > 0) {
|
|
|
return QUEUE_HIGHEST_PRIORITY;
|
|
|
}
|
|
|
+ if (readOnlyReplicas > 0) {
|
|
|
+ // only has read-only replicas, highest risk
|
|
|
+ // since the read-only replicas may go down all together.
|
|
|
+ return QUEUE_HIGHEST_PRIORITY;
|
|
|
+ }
|
|
|
//all we have are corrupt blocks
|
|
|
return QUEUE_WITH_CORRUPT_BLOCKS;
|
|
|
} else if (curReplicas == 1) {
|
|
@@ -218,11 +226,12 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
*/
|
|
|
synchronized boolean add(BlockInfo block,
|
|
|
int curReplicas,
|
|
|
+ int readOnlyReplicas,
|
|
|
int decomissionedReplicas,
|
|
|
int expectedReplicas) {
|
|
|
assert curReplicas >= 0 : "Negative replicas!";
|
|
|
- int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
|
|
|
- expectedReplicas);
|
|
|
+ final int priLevel = getPriority(block, curReplicas, readOnlyReplicas,
|
|
|
+ decomissionedReplicas, expectedReplicas);
|
|
|
if(priorityQueues.get(priLevel).add(block)) {
|
|
|
if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
|
|
|
expectedReplicas == 1) {
|
|
@@ -242,11 +251,11 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
/** remove a block from a under replication queue */
|
|
|
synchronized boolean remove(BlockInfo block,
|
|
|
int oldReplicas,
|
|
|
+ int oldReadOnlyReplicas,
|
|
|
int decommissionedReplicas,
|
|
|
int oldExpectedReplicas) {
|
|
|
- int priLevel = getPriority(block, oldReplicas,
|
|
|
- decommissionedReplicas,
|
|
|
- oldExpectedReplicas);
|
|
|
+ final int priLevel = getPriority(block, oldReplicas, oldReadOnlyReplicas,
|
|
|
+ decommissionedReplicas, oldExpectedReplicas);
|
|
|
boolean removedBlock = remove(block, priLevel);
|
|
|
if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
|
|
|
oldExpectedReplicas == 1 &&
|
|
@@ -285,10 +294,10 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
// Try to remove the block from all queues if the block was
|
|
|
// not found in the queue for the given priority level.
|
|
|
for (int i = 0; i < LEVEL; i++) {
|
|
|
- if (priorityQueues.get(i).remove(block)) {
|
|
|
+ if (i != priLevel && priorityQueues.get(i).remove(block)) {
|
|
|
NameNode.blockStateChangeLog.debug(
|
|
|
"BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block" +
|
|
|
- " {} from priority queue {}", block, priLevel);
|
|
|
+ " {} from priority queue {}", block, i);
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
@@ -313,15 +322,15 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
* @param expectedReplicasDelta the change in the expected replica count from before
|
|
|
*/
|
|
|
synchronized void update(BlockInfo block, int curReplicas,
|
|
|
- int decommissionedReplicas,
|
|
|
+ int readOnlyReplicas, int decommissionedReplicas,
|
|
|
int curExpectedReplicas,
|
|
|
int curReplicasDelta, int expectedReplicasDelta) {
|
|
|
int oldReplicas = curReplicas-curReplicasDelta;
|
|
|
int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
|
|
|
- int curPri = getPriority(block, curReplicas, decommissionedReplicas,
|
|
|
- curExpectedReplicas);
|
|
|
- int oldPri = getPriority(block, oldReplicas, decommissionedReplicas,
|
|
|
- oldExpectedReplicas);
|
|
|
+ int curPri = getPriority(block, curReplicas, readOnlyReplicas,
|
|
|
+ decommissionedReplicas, curExpectedReplicas);
|
|
|
+ int oldPri = getPriority(block, oldReplicas, readOnlyReplicas,
|
|
|
+ decommissionedReplicas, oldExpectedReplicas);
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " +
|
|
|
block +
|
|
@@ -371,143 +380,69 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
|
|
|
* @return Return a list of block lists to be replicated. The block list index
|
|
|
* represents its replication priority.
|
|
|
*/
|
|
|
- public synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
|
|
|
+ synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
|
|
|
int blocksToProcess) {
|
|
|
- // initialize data structure for the return value
|
|
|
- List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
|
|
|
- for (int i = 0; i < LEVEL; i++) {
|
|
|
- blocksToReplicate.add(new ArrayList<BlockInfo>());
|
|
|
- }
|
|
|
-
|
|
|
- if (size() == 0) { // There are no blocks to collect.
|
|
|
- return blocksToReplicate;
|
|
|
- }
|
|
|
+ final List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
|
|
|
|
|
|
- int blockCount = 0;
|
|
|
- for (int priority = 0; priority < LEVEL; priority++) {
|
|
|
+ int count = 0;
|
|
|
+ int priority = 0;
|
|
|
+ for (; count < blocksToProcess && priority < LEVEL; priority++) {
|
|
|
+ if (priority == QUEUE_WITH_CORRUPT_BLOCKS) {
|
|
|
+ // do not choose corrupted blocks.
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
// Go through all blocks that need replications with current priority.
|
|
|
- BlockIterator neededReplicationsIterator = iterator(priority);
|
|
|
// Set the iterator to the first unprocessed block at this priority level.
|
|
|
- neededReplicationsIterator.setToBookmark();
|
|
|
-
|
|
|
- blocksToProcess = Math.min(blocksToProcess, size());
|
|
|
-
|
|
|
- if (blockCount == blocksToProcess) {
|
|
|
- break; // break if already expected blocks are obtained
|
|
|
- }
|
|
|
-
|
|
|
+ final Iterator<BlockInfo> i = priorityQueues.get(priority).getBookmark();
|
|
|
+ final List<BlockInfo> blocks = new LinkedList<>();
|
|
|
+ blocksToReplicate.add(blocks);
|
|
|
// Loop through all remaining blocks in the list.
|
|
|
- while (blockCount < blocksToProcess
|
|
|
- && neededReplicationsIterator.hasNext()) {
|
|
|
- BlockInfo block = neededReplicationsIterator.next();
|
|
|
- blocksToReplicate.get(priority).add(block);
|
|
|
- blockCount++;
|
|
|
+ for(; count < blocksToProcess && i.hasNext(); count++) {
|
|
|
+ blocks.add(i.next());
|
|
|
}
|
|
|
-
|
|
|
- if (!neededReplicationsIterator.hasNext()
|
|
|
- && neededReplicationsIterator.getPriority() == LEVEL - 1) {
|
|
|
- // Reset all priorities' bookmarks to the beginning because there were
|
|
|
- // no recently added blocks in any list.
|
|
|
- for (int i = 0; i < LEVEL; i++) {
|
|
|
- this.priorityQueues.get(i).resetBookmark();
|
|
|
- }
|
|
|
- break;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (priority == LEVEL) {
|
|
|
+ // Reset all bookmarks because there were no recently added blocks.
|
|
|
+ for (LightWeightLinkedSet<BlockInfo> q : priorityQueues) {
|
|
|
+ q.resetBookmark();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
return blocksToReplicate;
|
|
|
}
|
|
|
|
|
|
/** returns an iterator of all blocks in a given priority queue */
|
|
|
- synchronized BlockIterator iterator(int level) {
|
|
|
- return new BlockIterator(level);
|
|
|
+ synchronized Iterator<BlockInfo> iterator(int level) {
|
|
|
+ return priorityQueues.get(level).iterator();
|
|
|
}
|
|
|
|
|
|
/** return an iterator of all the under replication blocks */
|
|
|
@Override
|
|
|
- public synchronized BlockIterator iterator() {
|
|
|
- return new BlockIterator();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * An iterator over blocks.
|
|
|
- */
|
|
|
- class BlockIterator implements Iterator<BlockInfo> {
|
|
|
- private int level;
|
|
|
- private boolean isIteratorForLevel = false;
|
|
|
- private final List<Iterator<BlockInfo>> iterators = new ArrayList<>();
|
|
|
-
|
|
|
- /**
|
|
|
- * Construct an iterator over all queues.
|
|
|
- */
|
|
|
- private BlockIterator() {
|
|
|
- level=0;
|
|
|
- for(int i=0; i<LEVEL; i++) {
|
|
|
- iterators.add(priorityQueues.get(i).iterator());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Constrict an iterator for a single queue level
|
|
|
- * @param l the priority level to iterate over
|
|
|
- */
|
|
|
- private BlockIterator(int l) {
|
|
|
- level = l;
|
|
|
- isIteratorForLevel = true;
|
|
|
- iterators.add(priorityQueues.get(level).iterator());
|
|
|
- }
|
|
|
-
|
|
|
- private void update() {
|
|
|
- if (isIteratorForLevel) {
|
|
|
- return;
|
|
|
- }
|
|
|
- while(level< LEVEL-1 && !iterators.get(level).hasNext()) {
|
|
|
- level++;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public BlockInfo next() {
|
|
|
- if (isIteratorForLevel) {
|
|
|
- return iterators.get(0).next();
|
|
|
+ public synchronized Iterator<BlockInfo> iterator() {
|
|
|
+ final Iterator<LightWeightLinkedSet<BlockInfo>> q = priorityQueues.iterator();
|
|
|
+ return new Iterator<BlockInfo>() {
|
|
|
+ private Iterator<BlockInfo> b = q.next().iterator();
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public BlockInfo next() {
|
|
|
+ hasNext();
|
|
|
+ return b.next();
|
|
|
}
|
|
|
- update();
|
|
|
- return iterators.get(level).next();
|
|
|
- }
|
|
|
|
|
|
- @Override
|
|
|
- public boolean hasNext() {
|
|
|
- if (isIteratorForLevel) {
|
|
|
- return iterators.get(0).hasNext();
|
|
|
- }
|
|
|
- update();
|
|
|
- return iterators.get(level).hasNext();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void remove() {
|
|
|
- if (isIteratorForLevel) {
|
|
|
- iterators.get(0).remove();
|
|
|
- } else {
|
|
|
- iterators.get(level).remove();
|
|
|
+ @Override
|
|
|
+ public boolean hasNext() {
|
|
|
+ for(; !b.hasNext() && q.hasNext(); ) {
|
|
|
+ b = q.next().iterator();
|
|
|
+ }
|
|
|
+ return b.hasNext();
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- int getPriority() {
|
|
|
- return level;
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Sets iterator(s) to bookmarked elements.
|
|
|
- */
|
|
|
- private synchronized void setToBookmark() {
|
|
|
- if (this.isIteratorForLevel) {
|
|
|
- this.iterators.set(0, priorityQueues.get(this.level)
|
|
|
- .getBookmark());
|
|
|
- } else {
|
|
|
- for(int i=0; i<LEVEL; i++) {
|
|
|
- this.iterators.set(i, priorityQueues.get(i).getBookmark());
|
|
|
- }
|
|
|
+ @Override
|
|
|
+ public void remove() {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
}
|
|
|
- }
|
|
|
+ };
|
|
|
}
|
|
|
}
|