Skip to content

Commit 8dfcd95

Browse files
committed
HDFS-11634. Optimize BlockIterator when interating starts in the middle. Contributed by Konstantin V Shvachko.
1 parent c0ca785 commit 8dfcd95

File tree

4 files changed

+115
-16
lines changed

4 files changed

+115
-16
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1372,13 +1372,9 @@ public BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode,
13721372
if(numBlocks == 0) {
13731373
return new BlocksWithLocations(new BlockWithLocations[0]);
13741374
}
1375-
Iterator<BlockInfo> iter = node.getBlockIterator();
13761375
// starting from a random block
13771376
int startBlock = ThreadLocalRandom.current().nextInt(numBlocks);
1378-
// skip blocks
1379-
for(int i=0; i<startBlock; i++) {
1380-
iter.next();
1381-
}
1377+
Iterator<BlockInfo> iter = node.getBlockIterator(startBlock);
13821378
List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
13831379
long totalSize = 0;
13841380
BlockInfo curBlock;

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

Lines changed: 28 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -519,18 +519,35 @@ private static class BlockIterator implements Iterator<BlockInfo> {
519519
private int index = 0;
520520
private final List<Iterator<BlockInfo>> iterators;
521521

522-
private BlockIterator(final DatanodeStorageInfo... storages) {
522+
private BlockIterator(final int startBlock,
523+
final DatanodeStorageInfo... storages) {
524+
if(startBlock < 0) {
525+
throw new IllegalArgumentException(
526+
"Illegal value startBlock = " + startBlock);
527+
}
523528
List<Iterator<BlockInfo>> iterators = new ArrayList<>();
529+
int s = startBlock;
530+
int sumBlocks = 0;
524531
for (DatanodeStorageInfo e : storages) {
525-
iterators.add(e.getBlockIterator());
532+
int numBlocks = e.numBlocks();
533+
sumBlocks += numBlocks;
534+
if(sumBlocks <= startBlock) {
535+
s -= numBlocks;
536+
} else {
537+
iterators.add(e.getBlockIterator());
538+
}
526539
}
527540
this.iterators = Collections.unmodifiableList(iterators);
541+
// skip to the storage containing startBlock
542+
for(; s > 0 && hasNext(); s--) {
543+
next();
544+
}
528545
}
529546

530547
@Override
531548
public boolean hasNext() {
532549
update();
533-
return !iterators.isEmpty() && iterators.get(index).hasNext();
550+
return index < iterators.size() && iterators.get(index).hasNext();
534551
}
535552

536553
@Override
@@ -552,7 +569,14 @@ private void update() {
552569
}
553570

554571
Iterator<BlockInfo> getBlockIterator() {
555-
return new BlockIterator(getStorageInfos());
572+
return getBlockIterator(0);
573+
}
574+
575+
/**
576+
* Get iterator, which starts iterating from the specified block.
577+
*/
578+
Iterator<BlockInfo> getBlockIterator(final int startBlock) {
579+
return new BlockIterator(startBlock, getStorageInfos());
556580
}
557581

558582
void incrementPendingReplicationWithoutTargets() {

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

Lines changed: 70 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.io.IOException;
2323
import java.net.InetSocketAddress;
2424
import java.util.HashMap;
25+
import java.util.Iterator;
2526
import java.util.List;
2627
import java.util.Map;
2728
import java.util.Random;
@@ -38,9 +39,13 @@
3839
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
3940
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
4041
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
42+
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
43+
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
4144
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
45+
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
4246
import org.apache.hadoop.hdfs.server.datanode.DataNode;
4347
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
48+
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
4449
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
4550
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
4651
import org.apache.hadoop.ipc.RemoteException;
@@ -182,25 +187,27 @@ public void testGetBlocks() throws Exception {
182187
CONF.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY,
183188
DEFAULT_BLOCK_SIZE);
184189

185-
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
186-
REPLICATION_FACTOR).build();
190+
MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
191+
.numDataNodes(REPLICATION_FACTOR)
192+
.storagesPerDatanode(4)
193+
.build();
187194
try {
188195
cluster.waitActive();
189196
// the third block will not be visible to getBlocks
190-
long fileLen = 2 * DEFAULT_BLOCK_SIZE + 1;
197+
long fileLen = 12 * DEFAULT_BLOCK_SIZE + 1;
191198
DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/tmp.txt"),
192199
fileLen, REPLICATION_FACTOR, 0L);
193200

194201
// get blocks & data nodes
195202
List<LocatedBlock> locatedBlocks;
196203
DatanodeInfo[] dataNodes = null;
197204
boolean notWritten;
205+
final DFSClient dfsclient = new DFSClient(
206+
DFSUtilClient.getNNAddress(CONF), CONF);
198207
do {
199-
final DFSClient dfsclient = new DFSClient(
200-
DFSUtilClient.getNNAddress(CONF), CONF);
201208
locatedBlocks = dfsclient.getNamenode()
202209
.getBlockLocations("/tmp.txt", 0, fileLen).getLocatedBlocks();
203-
assertEquals(3, locatedBlocks.size());
210+
assertEquals(13, locatedBlocks.size());
204211
notWritten = false;
205212
for (int i = 0; i < 2; i++) {
206213
dataNodes = locatedBlocks.get(i).getLocations();
@@ -214,6 +221,7 @@ public void testGetBlocks() throws Exception {
214221
}
215222
}
216223
} while (notWritten);
224+
dfsclient.close();
217225

218226
// get RPC client to namenode
219227
InetSocketAddress addr = new InetSocketAddress("localhost",
@@ -224,7 +232,7 @@ public void testGetBlocks() throws Exception {
224232
// get blocks of size fileLen from dataNodes[0]
225233
BlockWithLocations[] locs;
226234
locs = namenode.getBlocks(dataNodes[0], fileLen).getBlocks();
227-
assertEquals(locs.length, 2);
235+
assertEquals(locs.length, 12);
228236
assertEquals(locs[0].getStorageIDs().length, 2);
229237
assertEquals(locs[1].getStorageIDs().length, 2);
230238

@@ -247,6 +255,8 @@ public void testGetBlocks() throws Exception {
247255
// get blocks of size BlockSize from a non-existent datanode
248256
DatanodeInfo info = DFSTestUtil.getDatanodeInfo("1.2.3.4");
249257
getBlocksWithException(namenode, info, 2);
258+
259+
testBlockIterator(cluster);
250260
} finally {
251261
cluster.shutdown();
252262
}
@@ -264,6 +274,59 @@ private void getBlocksWithException(NamenodeProtocol namenode,
264274
assertTrue(getException);
265275
}
266276

277+
/**
278+
* BlockIterator iterates over all blocks belonging to DatanodeDescriptor
279+
* through multiple storages.
280+
* The test verifies that BlockIterator can be set to start iterating from
281+
* a particular starting block index.
282+
*/
283+
void testBlockIterator(MiniDFSCluster cluster) {
284+
FSNamesystem ns = cluster.getNamesystem();
285+
String dId = cluster.getDataNodes().get(0).getDatanodeUuid();
286+
DatanodeDescriptor dnd = BlockManagerTestUtil.getDatanode(ns, dId);
287+
DatanodeStorageInfo[] storages = dnd.getStorageInfos();
288+
assertEquals("DataNode should have 4 storages", 4, storages.length);
289+
290+
Iterator<BlockInfo> dnBlockIt = null;
291+
// check illegal start block number
292+
try {
293+
dnBlockIt = BlockManagerTestUtil.getBlockIterator(
294+
cluster.getNamesystem(), dId, -1);
295+
assertTrue("Should throw IllegalArgumentException", false);
296+
} catch(IllegalArgumentException ei) {
297+
// as expected
298+
}
299+
assertNull("Iterator should be null", dnBlockIt);
300+
301+
// form an array of all DataNode blocks
302+
int numBlocks = dnd.numBlocks();
303+
BlockInfo[] allBlocks = new BlockInfo[numBlocks];
304+
int idx = 0;
305+
for(DatanodeStorageInfo s : storages) {
306+
Iterator<BlockInfo> storageBlockIt =
307+
BlockManagerTestUtil.getBlockIterator(s);
308+
while(storageBlockIt.hasNext()) {
309+
allBlocks[idx++] = storageBlockIt.next();
310+
}
311+
}
312+
313+
// check iterator for every block as a starting point
314+
for(int i = 0; i < allBlocks.length; i++) {
315+
// create iterator starting from i
316+
dnBlockIt = BlockManagerTestUtil.getBlockIterator(ns, dId, i);
317+
assertTrue("Block iterator should have next block", dnBlockIt.hasNext());
318+
// check iterator lists blocks in the desired order
319+
for(int j = i; j < allBlocks.length; j++) {
320+
assertEquals("Wrong block order", allBlocks[j], dnBlockIt.next());
321+
}
322+
}
323+
324+
// check start block number larger than numBlocks in the DataNode
325+
dnBlockIt = BlockManagerTestUtil.getBlockIterator(
326+
ns, dId, allBlocks.length + 1);
327+
assertFalse("Iterator should not have next block", dnBlockIt.hasNext());
328+
}
329+
267330
@Test
268331
public void testBlockKey() {
269332
Map<Block, Long> map = new HashMap<Block, Long>();

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.util.ArrayList;
2222
import java.util.Collection;
2323
import java.util.HashSet;
24+
import java.util.Iterator;
2425
import java.util.Set;
2526
import java.util.concurrent.ExecutionException;
2627

@@ -55,6 +56,21 @@ public static DatanodeDescriptor getDatanode(final FSNamesystem ns,
5556
}
5657
}
5758

59+
public static Iterator<BlockInfo> getBlockIterator(final FSNamesystem ns,
60+
final String storageID, final int startBlock) {
61+
ns.readLock();
62+
try {
63+
DatanodeDescriptor dn =
64+
ns.getBlockManager().getDatanodeManager().getDatanode(storageID);
65+
return dn.getBlockIterator(startBlock);
66+
} finally {
67+
ns.readUnlock();
68+
}
69+
}
70+
71+
public static Iterator<BlockInfo> getBlockIterator(DatanodeStorageInfo s) {
72+
return s.getBlockIterator();
73+
}
5874

5975
/**
6076
* Refresh block queue counts on the name-node.

0 commit comments

Comments
 (0)