001 /** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019 package org.apache.hadoop.mapreduce.lib.input; 020 021 import java.io.IOException; 022 import java.util.ArrayList; 023 import java.util.Collection; 024 import java.util.LinkedList; 025 import java.util.Collections; 026 import java.util.LinkedHashSet; 027 import java.util.HashSet; 028 import java.util.List; 029 import java.util.HashMap; 030 import java.util.Set; 031 import java.util.Iterator; 032 import java.util.Map; 033 import java.util.Map.Entry; 034 035 import org.apache.commons.logging.Log; 036 import org.apache.commons.logging.LogFactory; 037 import org.apache.hadoop.classification.InterfaceAudience; 038 import org.apache.hadoop.classification.InterfaceStability; 039 import org.apache.hadoop.conf.Configuration; 040 import org.apache.hadoop.fs.FileSystem; 041 import org.apache.hadoop.fs.FileUtil; 042 import org.apache.hadoop.fs.Path; 043 import org.apache.hadoop.fs.BlockLocation; 044 import org.apache.hadoop.fs.FileStatus; 045 import org.apache.hadoop.fs.PathFilter; 046 import org.apache.hadoop.io.compress.CompressionCodec; 047 import org.apache.hadoop.io.compress.CompressionCodecFactory; 048 import org.apache.hadoop.io.compress.SplittableCompressionCodec; 049 import org.apache.hadoop.mapreduce.InputFormat; 050 import org.apache.hadoop.mapreduce.InputSplit; 051 import org.apache.hadoop.mapreduce.JobContext; 052 import org.apache.hadoop.mapreduce.RecordReader; 053 import org.apache.hadoop.mapreduce.TaskAttemptContext; 054 import org.apache.hadoop.net.NodeBase; 055 import org.apache.hadoop.net.NetworkTopology; 056 057 import com.google.common.annotations.VisibleForTesting; 058 import com.google.common.collect.HashMultiset; 059 import com.google.common.collect.Multiset; 060 061 /** 062 * An abstract {@link InputFormat} that returns {@link CombineFileSplit}'s in 063 * {@link InputFormat#getSplits(JobContext)} method. 064 * 065 * Splits are constructed from the files under the input paths. 066 * A split cannot have files from different pools. 067 * Each split returned may contain blocks from different files. 068 * If a maxSplitSize is specified, then blocks on the same node are 069 * combined to form a single split. Blocks that are left over are 070 * then combined with other blocks in the same rack. 071 * If maxSplitSize is not specified, then blocks from the same rack 072 * are combined in a single split; no attempt is made to create 073 * node-local splits. 074 * If the maxSplitSize is equal to the block size, then this class 075 * is similar to the default splitting behavior in Hadoop: each 076 * block is a locally processed split. 077 * Subclasses implement 078 * {@link InputFormat#createRecordReader(InputSplit, TaskAttemptContext)} 079 * to construct <code>RecordReader</code>'s for 080 * <code>CombineFileSplit</code>'s. 081 * 082 * @see CombineFileSplit 083 */ 084 @InterfaceAudience.Public 085 @InterfaceStability.Stable 086 public abstract class CombineFileInputFormat<K, V> 087 extends FileInputFormat<K, V> { 088 089 private static final Log LOG = LogFactory.getLog(CombineFileInputFormat.class); 090 091 public static final String SPLIT_MINSIZE_PERNODE = 092 "mapreduce.input.fileinputformat.split.minsize.per.node"; 093 public static final String SPLIT_MINSIZE_PERRACK = 094 "mapreduce.input.fileinputformat.split.minsize.per.rack"; 095 // ability to limit the size of a single split 096 private long maxSplitSize = 0; 097 private long minSplitSizeNode = 0; 098 private long minSplitSizeRack = 0; 099 100 // A pool of input paths filters. A split cannot have blocks from files 101 // across multiple pools. 102 private ArrayList<MultiPathFilter> pools = new ArrayList<MultiPathFilter>(); 103 104 // mapping from a rack name to the set of Nodes in the rack 105 private HashMap<String, Set<String>> rackToNodes = 106 new HashMap<String, Set<String>>(); 107 /** 108 * Specify the maximum size (in bytes) of each split. Each split is 109 * approximately equal to the specified size. 110 */ 111 protected void setMaxSplitSize(long maxSplitSize) { 112 this.maxSplitSize = maxSplitSize; 113 } 114 115 /** 116 * Specify the minimum size (in bytes) of each split per node. 117 * This applies to data that is left over after combining data on a single 118 * node into splits that are of maximum size specified by maxSplitSize. 119 * This leftover data will be combined into its own split if its size 120 * exceeds minSplitSizeNode. 121 */ 122 protected void setMinSplitSizeNode(long minSplitSizeNode) { 123 this.minSplitSizeNode = minSplitSizeNode; 124 } 125 126 /** 127 * Specify the minimum size (in bytes) of each split per rack. 128 * This applies to data that is left over after combining data on a single 129 * rack into splits that are of maximum size specified by maxSplitSize. 130 * This leftover data will be combined into its own split if its size 131 * exceeds minSplitSizeRack. 132 */ 133 protected void setMinSplitSizeRack(long minSplitSizeRack) { 134 this.minSplitSizeRack = minSplitSizeRack; 135 } 136 137 /** 138 * Create a new pool and add the filters to it. 139 * A split cannot have files from different pools. 140 */ 141 protected void createPool(List<PathFilter> filters) { 142 pools.add(new MultiPathFilter(filters)); 143 } 144 145 /** 146 * Create a new pool and add the filters to it. 147 * A pathname can satisfy any one of the specified filters. 148 * A split cannot have files from different pools. 149 */ 150 protected void createPool(PathFilter... filters) { 151 MultiPathFilter multi = new MultiPathFilter(); 152 for (PathFilter f: filters) { 153 multi.add(f); 154 } 155 pools.add(multi); 156 } 157 158 @Override 159 protected boolean isSplitable(JobContext context, Path file) { 160 final CompressionCodec codec = 161 new CompressionCodecFactory(context.getConfiguration()).getCodec(file); 162 if (null == codec) { 163 return true; 164 } 165 return codec instanceof SplittableCompressionCodec; 166 } 167 168 /** 169 * default constructor 170 */ 171 public CombineFileInputFormat() { 172 } 173 174 @Override 175 public List<InputSplit> getSplits(JobContext job) 176 throws IOException { 177 long minSizeNode = 0; 178 long minSizeRack = 0; 179 long maxSize = 0; 180 Configuration conf = job.getConfiguration(); 181 182 // the values specified by setxxxSplitSize() takes precedence over the 183 // values that might have been specified in the config 184 if (minSplitSizeNode != 0) { 185 minSizeNode = minSplitSizeNode; 186 } else { 187 minSizeNode = conf.getLong(SPLIT_MINSIZE_PERNODE, 0); 188 } 189 if (minSplitSizeRack != 0) { 190 minSizeRack = minSplitSizeRack; 191 } else { 192 minSizeRack = conf.getLong(SPLIT_MINSIZE_PERRACK, 0); 193 } 194 if (maxSplitSize != 0) { 195 maxSize = maxSplitSize; 196 } else { 197 maxSize = conf.getLong("mapreduce.input.fileinputformat.split.maxsize", 0); 198 // If maxSize is not configured, a single split will be generated per 199 // node. 200 } 201 if (minSizeNode != 0 && maxSize != 0 && minSizeNode > maxSize) { 202 throw new IOException("Minimum split size pernode " + minSizeNode + 203 " cannot be larger than maximum split size " + 204 maxSize); 205 } 206 if (minSizeRack != 0 && maxSize != 0 && minSizeRack > maxSize) { 207 throw new IOException("Minimum split size per rack" + minSizeRack + 208 " cannot be larger than maximum split size " + 209 maxSize); 210 } 211 if (minSizeRack != 0 && minSizeNode > minSizeRack) { 212 throw new IOException("Minimum split size per node" + minSizeNode + 213 " cannot be smaller than minimum split " + 214 "size per rack " + minSizeRack); 215 } 216 217 // all the files in input set 218 Path[] paths = FileUtil.stat2Paths( 219 listStatus(job).toArray(new FileStatus[0])); 220 List<InputSplit> splits = new ArrayList<InputSplit>(); 221 if (paths.length == 0) { 222 return splits; 223 } 224 225 // Convert them to Paths first. This is a costly operation and 226 // we should do it first, otherwise we will incur doing it multiple 227 // times, one time each for each pool in the next loop. 228 List<Path> newpaths = new LinkedList<Path>(); 229 for (int i = 0; i < paths.length; i++) { 230 FileSystem fs = paths[i].getFileSystem(conf); 231 Path p = fs.makeQualified(paths[i]); 232 newpaths.add(p); 233 } 234 235 // In one single iteration, process all the paths in a single pool. 236 // Processing one pool at a time ensures that a split contains paths 237 // from a single pool only. 238 for (MultiPathFilter onepool : pools) { 239 ArrayList<Path> myPaths = new ArrayList<Path>(); 240 241 // pick one input path. If it matches all the filters in a pool, 242 // add it to the output set 243 for (Iterator<Path> iter = newpaths.iterator(); iter.hasNext();) { 244 Path p = iter.next(); 245 if (onepool.accept(p)) { 246 myPaths.add(p); // add it to my output set 247 iter.remove(); 248 } 249 } 250 // create splits for all files in this pool. 251 getMoreSplits(job, myPaths.toArray(new Path[myPaths.size()]), 252 maxSize, minSizeNode, minSizeRack, splits); 253 } 254 255 // create splits for all files that are not in any pool. 256 getMoreSplits(job, newpaths.toArray(new Path[newpaths.size()]), 257 maxSize, minSizeNode, minSizeRack, splits); 258 259 // free up rackToNodes map 260 rackToNodes.clear(); 261 return splits; 262 } 263 264 /** 265 * Return all the splits in the specified set of paths 266 */ 267 private void getMoreSplits(JobContext job, Path[] paths, 268 long maxSize, long minSizeNode, long minSizeRack, 269 List<InputSplit> splits) 270 throws IOException { 271 Configuration conf = job.getConfiguration(); 272 273 // all blocks for all the files in input set 274 OneFileInfo[] files; 275 276 // mapping from a rack name to the list of blocks it has 277 HashMap<String, List<OneBlockInfo>> rackToBlocks = 278 new HashMap<String, List<OneBlockInfo>>(); 279 280 // mapping from a block to the nodes on which it has replicas 281 HashMap<OneBlockInfo, String[]> blockToNodes = 282 new HashMap<OneBlockInfo, String[]>(); 283 284 // mapping from a node to the list of blocks that it contains 285 HashMap<String, Set<OneBlockInfo>> nodeToBlocks = 286 new HashMap<String, Set<OneBlockInfo>>(); 287 288 files = new OneFileInfo[paths.length]; 289 if (paths.length == 0) { 290 return; 291 } 292 293 // populate all the blocks for all files 294 long totLength = 0; 295 for (int i = 0; i < paths.length; i++) { 296 files[i] = new OneFileInfo(paths[i], conf, isSplitable(job, paths[i]), 297 rackToBlocks, blockToNodes, nodeToBlocks, 298 rackToNodes, maxSize); 299 totLength += files[i].getLength(); 300 } 301 createSplits(nodeToBlocks, blockToNodes, rackToBlocks, totLength, 302 maxSize, minSizeNode, minSizeRack, splits); 303 } 304 305 @VisibleForTesting 306 void createSplits(Map<String, Set<OneBlockInfo>> nodeToBlocks, 307 Map<OneBlockInfo, String[]> blockToNodes, 308 Map<String, List<OneBlockInfo>> rackToBlocks, 309 long totLength, 310 long maxSize, 311 long minSizeNode, 312 long minSizeRack, 313 List<InputSplit> splits 314 ) { 315 ArrayList<OneBlockInfo> validBlocks = new ArrayList<OneBlockInfo>(); 316 long curSplitSize = 0; 317 318 int totalNodes = nodeToBlocks.size(); 319 long totalLength = totLength; 320 321 Multiset<String> splitsPerNode = HashMultiset.create(); 322 Set<String> completedNodes = new HashSet<String>(); 323 324 while(true) { 325 // it is allowed for maxSize to be 0. Disable smoothing load for such cases 326 327 // process all nodes and create splits that are local to a node. Generate 328 // one split per node iteration, and walk over nodes multiple times to 329 // distribute the splits across nodes. 330 for (Iterator<Map.Entry<String, Set<OneBlockInfo>>> iter = nodeToBlocks 331 .entrySet().iterator(); iter.hasNext();) { 332 Map.Entry<String, Set<OneBlockInfo>> one = iter.next(); 333 334 String node = one.getKey(); 335 336 // Skip the node if it has previously been marked as completed. 337 if (completedNodes.contains(node)) { 338 continue; 339 } 340 341 Set<OneBlockInfo> blocksInCurrentNode = one.getValue(); 342 343 // for each block, copy it into validBlocks. Delete it from 344 // blockToNodes so that the same block does not appear in 345 // two different splits. 346 Iterator<OneBlockInfo> oneBlockIter = blocksInCurrentNode.iterator(); 347 while (oneBlockIter.hasNext()) { 348 OneBlockInfo oneblock = oneBlockIter.next(); 349 350 // Remove all blocks which may already have been assigned to other 351 // splits. 352 if(!blockToNodes.containsKey(oneblock)) { 353 oneBlockIter.remove(); 354 continue; 355 } 356 357 validBlocks.add(oneblock); 358 blockToNodes.remove(oneblock); 359 curSplitSize += oneblock.length; 360 361 // if the accumulated split size exceeds the maximum, then 362 // create this split. 363 if (maxSize != 0 && curSplitSize >= maxSize) { 364 // create an input split and add it to the splits array 365 addCreatedSplit(splits, Collections.singleton(node), validBlocks); 366 totalLength -= curSplitSize; 367 curSplitSize = 0; 368 369 splitsPerNode.add(node); 370 371 // Remove entries from blocksInNode so that we don't walk these 372 // again. 373 blocksInCurrentNode.removeAll(validBlocks); 374 validBlocks.clear(); 375 376 // Done creating a single split for this node. Move on to the next 377 // node so that splits are distributed across nodes. 378 break; 379 } 380 381 } 382 if (validBlocks.size() != 0) { 383 // This implies that the last few blocks (or all in case maxSize=0) 384 // were not part of a split. The node is complete. 385 386 // if there were any blocks left over and their combined size is 387 // larger than minSplitNode, then combine them into one split. 388 // Otherwise add them back to the unprocessed pool. It is likely 389 // that they will be combined with other blocks from the 390 // same rack later on. 391 // This condition also kicks in when max split size is not set. All 392 // blocks on a node will be grouped together into a single split. 393 if (minSizeNode != 0 && curSplitSize >= minSizeNode 394 && splitsPerNode.count(node) == 0) { 395 // haven't created any split on this machine. so its ok to add a 396 // smaller one for parallelism. Otherwise group it in the rack for 397 // balanced size create an input split and add it to the splits 398 // array 399 addCreatedSplit(splits, Collections.singleton(node), validBlocks); 400 totalLength -= curSplitSize; 401 splitsPerNode.add(node); 402 // Remove entries from blocksInNode so that we don't walk this again. 403 blocksInCurrentNode.removeAll(validBlocks); 404 // The node is done. This was the last set of blocks for this node. 405 } else { 406 // Put the unplaced blocks back into the pool for later rack-allocation. 407 for (OneBlockInfo oneblock : validBlocks) { 408 blockToNodes.put(oneblock, oneblock.hosts); 409 } 410 } 411 validBlocks.clear(); 412 curSplitSize = 0; 413 completedNodes.add(node); 414 } else { // No in-flight blocks. 415 if (blocksInCurrentNode.size() == 0) { 416 // Node is done. All blocks were fit into node-local splits. 417 completedNodes.add(node); 418 } // else Run through the node again. 419 } 420 } 421 422 // Check if node-local assignments are complete. 423 if (completedNodes.size() == totalNodes || totalLength == 0) { 424 // All nodes have been walked over and marked as completed or all blocks 425 // have been assigned. The rest should be handled via rackLock assignment. 426 LOG.info("DEBUG: Terminated node allocation with : CompletedNodes: " 427 + completedNodes.size() + ", size left: " + totalLength); 428 break; 429 } 430 } 431 432 // if blocks in a rack are below the specified minimum size, then keep them 433 // in 'overflow'. After the processing of all racks is complete, these 434 // overflow blocks will be combined into splits. 435 ArrayList<OneBlockInfo> overflowBlocks = new ArrayList<OneBlockInfo>(); 436 Set<String> racks = new HashSet<String>(); 437 438 // Process all racks over and over again until there is no more work to do. 439 while (blockToNodes.size() > 0) { 440 441 // Create one split for this rack before moving over to the next rack. 442 // Come back to this rack after creating a single split for each of the 443 // remaining racks. 444 // Process one rack location at a time, Combine all possible blocks that 445 // reside on this rack as one split. (constrained by minimum and maximum 446 // split size). 447 448 // iterate over all racks 449 for (Iterator<Map.Entry<String, List<OneBlockInfo>>> iter = 450 rackToBlocks.entrySet().iterator(); iter.hasNext();) { 451 452 Map.Entry<String, List<OneBlockInfo>> one = iter.next(); 453 racks.add(one.getKey()); 454 List<OneBlockInfo> blocks = one.getValue(); 455 456 // for each block, copy it into validBlocks. Delete it from 457 // blockToNodes so that the same block does not appear in 458 // two different splits. 459 boolean createdSplit = false; 460 for (OneBlockInfo oneblock : blocks) { 461 if (blockToNodes.containsKey(oneblock)) { 462 validBlocks.add(oneblock); 463 blockToNodes.remove(oneblock); 464 curSplitSize += oneblock.length; 465 466 // if the accumulated split size exceeds the maximum, then 467 // create this split. 468 if (maxSize != 0 && curSplitSize >= maxSize) { 469 // create an input split and add it to the splits array 470 addCreatedSplit(splits, getHosts(racks), validBlocks); 471 createdSplit = true; 472 break; 473 } 474 } 475 } 476 477 // if we created a split, then just go to the next rack 478 if (createdSplit) { 479 curSplitSize = 0; 480 validBlocks.clear(); 481 racks.clear(); 482 continue; 483 } 484 485 if (!validBlocks.isEmpty()) { 486 if (minSizeRack != 0 && curSplitSize >= minSizeRack) { 487 // if there is a minimum size specified, then create a single split 488 // otherwise, store these blocks into overflow data structure 489 addCreatedSplit(splits, getHosts(racks), validBlocks); 490 } else { 491 // There were a few blocks in this rack that 492 // remained to be processed. Keep them in 'overflow' block list. 493 // These will be combined later. 494 overflowBlocks.addAll(validBlocks); 495 } 496 } 497 curSplitSize = 0; 498 validBlocks.clear(); 499 racks.clear(); 500 } 501 } 502 503 assert blockToNodes.isEmpty(); 504 assert curSplitSize == 0; 505 assert validBlocks.isEmpty(); 506 assert racks.isEmpty(); 507 508 // Process all overflow blocks 509 for (OneBlockInfo oneblock : overflowBlocks) { 510 validBlocks.add(oneblock); 511 curSplitSize += oneblock.length; 512 513 // This might cause an exiting rack location to be re-added, 514 // but it should be ok. 515 for (int i = 0; i < oneblock.racks.length; i++) { 516 racks.add(oneblock.racks[i]); 517 } 518 519 // if the accumulated split size exceeds the maximum, then 520 // create this split. 521 if (maxSize != 0 && curSplitSize >= maxSize) { 522 // create an input split and add it to the splits array 523 addCreatedSplit(splits, getHosts(racks), validBlocks); 524 curSplitSize = 0; 525 validBlocks.clear(); 526 racks.clear(); 527 } 528 } 529 530 // Process any remaining blocks, if any. 531 if (!validBlocks.isEmpty()) { 532 addCreatedSplit(splits, getHosts(racks), validBlocks); 533 } 534 } 535 536 /** 537 * Create a single split from the list of blocks specified in validBlocks 538 * Add this new split into splitList. 539 */ 540 private void addCreatedSplit(List<InputSplit> splitList, 541 Collection<String> locations, 542 ArrayList<OneBlockInfo> validBlocks) { 543 // create an input split 544 Path[] fl = new Path[validBlocks.size()]; 545 long[] offset = new long[validBlocks.size()]; 546 long[] length = new long[validBlocks.size()]; 547 for (int i = 0; i < validBlocks.size(); i++) { 548 fl[i] = validBlocks.get(i).onepath; 549 offset[i] = validBlocks.get(i).offset; 550 length[i] = validBlocks.get(i).length; 551 } 552 // add this split to the list that is returned 553 CombineFileSplit thissplit = new CombineFileSplit(fl, offset, 554 length, locations.toArray(new String[0])); 555 splitList.add(thissplit); 556 } 557 558 /** 559 * This is not implemented yet. 560 */ 561 public abstract RecordReader<K, V> createRecordReader(InputSplit split, 562 TaskAttemptContext context) throws IOException; 563 564 /** 565 * information about one file from the File System 566 */ 567 @VisibleForTesting 568 static class OneFileInfo { 569 private long fileSize; // size of the file 570 private OneBlockInfo[] blocks; // all blocks in this file 571 572 OneFileInfo(Path path, Configuration conf, 573 boolean isSplitable, 574 HashMap<String, List<OneBlockInfo>> rackToBlocks, 575 HashMap<OneBlockInfo, String[]> blockToNodes, 576 HashMap<String, Set<OneBlockInfo>> nodeToBlocks, 577 HashMap<String, Set<String>> rackToNodes, 578 long maxSize) 579 throws IOException { 580 this.fileSize = 0; 581 582 // get block locations from file system 583 FileSystem fs = path.getFileSystem(conf); 584 FileStatus stat = fs.getFileStatus(path); 585 BlockLocation[] locations = fs.getFileBlockLocations(stat, 0, 586 stat.getLen()); 587 // create a list of all block and their locations 588 if (locations == null) { 589 blocks = new OneBlockInfo[0]; 590 } else { 591 592 if(locations.length == 0) { 593 locations = new BlockLocation[] { new BlockLocation() }; 594 } 595 596 if (!isSplitable) { 597 // if the file is not splitable, just create the one block with 598 // full file length 599 blocks = new OneBlockInfo[1]; 600 fileSize = stat.getLen(); 601 blocks[0] = new OneBlockInfo(path, 0, fileSize, locations[0] 602 .getHosts(), locations[0].getTopologyPaths()); 603 } else { 604 ArrayList<OneBlockInfo> blocksList = new ArrayList<OneBlockInfo>( 605 locations.length); 606 for (int i = 0; i < locations.length; i++) { 607 fileSize += locations[i].getLength(); 608 609 // each split can be a maximum of maxSize 610 long left = locations[i].getLength(); 611 long myOffset = locations[i].getOffset(); 612 long myLength = 0; 613 do { 614 if (maxSize == 0) { 615 myLength = left; 616 } else { 617 if (left > maxSize && left < 2 * maxSize) { 618 // if remainder is between max and 2*max - then 619 // instead of creating splits of size max, left-max we 620 // create splits of size left/2 and left/2. This is 621 // a heuristic to avoid creating really really small 622 // splits. 623 myLength = left / 2; 624 } else { 625 myLength = Math.min(maxSize, left); 626 } 627 } 628 OneBlockInfo oneblock = new OneBlockInfo(path, myOffset, 629 myLength, locations[i].getHosts(), locations[i] 630 .getTopologyPaths()); 631 left -= myLength; 632 myOffset += myLength; 633 634 blocksList.add(oneblock); 635 } while (left > 0); 636 } 637 blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]); 638 } 639 640 populateBlockInfo(blocks, rackToBlocks, blockToNodes, 641 nodeToBlocks, rackToNodes); 642 } 643 } 644 645 @VisibleForTesting 646 static void populateBlockInfo(OneBlockInfo[] blocks, 647 Map<String, List<OneBlockInfo>> rackToBlocks, 648 Map<OneBlockInfo, String[]> blockToNodes, 649 Map<String, Set<OneBlockInfo>> nodeToBlocks, 650 Map<String, Set<String>> rackToNodes) { 651 for (OneBlockInfo oneblock : blocks) { 652 // add this block to the block --> node locations map 653 blockToNodes.put(oneblock, oneblock.hosts); 654 655 // For blocks that do not have host/rack information, 656 // assign to default rack. 657 String[] racks = null; 658 if (oneblock.hosts.length == 0) { 659 racks = new String[]{NetworkTopology.DEFAULT_RACK}; 660 } else { 661 racks = oneblock.racks; 662 } 663 664 // add this block to the rack --> block map 665 for (int j = 0; j < racks.length; j++) { 666 String rack = racks[j]; 667 List<OneBlockInfo> blklist = rackToBlocks.get(rack); 668 if (blklist == null) { 669 blklist = new ArrayList<OneBlockInfo>(); 670 rackToBlocks.put(rack, blklist); 671 } 672 blklist.add(oneblock); 673 if (!racks[j].equals(NetworkTopology.DEFAULT_RACK)) { 674 // Add this host to rackToNodes map 675 addHostToRack(rackToNodes, racks[j], oneblock.hosts[j]); 676 } 677 } 678 679 // add this block to the node --> block map 680 for (int j = 0; j < oneblock.hosts.length; j++) { 681 String node = oneblock.hosts[j]; 682 Set<OneBlockInfo> blklist = nodeToBlocks.get(node); 683 if (blklist == null) { 684 blklist = new LinkedHashSet<OneBlockInfo>(); 685 nodeToBlocks.put(node, blklist); 686 } 687 blklist.add(oneblock); 688 } 689 } 690 } 691 692 long getLength() { 693 return fileSize; 694 } 695 696 OneBlockInfo[] getBlocks() { 697 return blocks; 698 } 699 } 700 701 /** 702 * information about one block from the File System 703 */ 704 @VisibleForTesting 705 static class OneBlockInfo { 706 Path onepath; // name of this file 707 long offset; // offset in file 708 long length; // length of this block 709 String[] hosts; // nodes on which this block resides 710 String[] racks; // network topology of hosts 711 712 OneBlockInfo(Path path, long offset, long len, 713 String[] hosts, String[] topologyPaths) { 714 this.onepath = path; 715 this.offset = offset; 716 this.hosts = hosts; 717 this.length = len; 718 assert (hosts.length == topologyPaths.length || 719 topologyPaths.length == 0); 720 721 // if the file system does not have any rack information, then 722 // use dummy rack location. 723 if (topologyPaths.length == 0) { 724 topologyPaths = new String[hosts.length]; 725 for (int i = 0; i < topologyPaths.length; i++) { 726 topologyPaths[i] = (new NodeBase(hosts[i], 727 NetworkTopology.DEFAULT_RACK)).toString(); 728 } 729 } 730 731 // The topology paths have the host name included as the last 732 // component. Strip it. 733 this.racks = new String[topologyPaths.length]; 734 for (int i = 0; i < topologyPaths.length; i++) { 735 this.racks[i] = (new NodeBase(topologyPaths[i])).getNetworkLocation(); 736 } 737 } 738 } 739 740 protected BlockLocation[] getFileBlockLocations( 741 FileSystem fs, FileStatus stat) throws IOException { 742 return fs.getFileBlockLocations(stat, 0, stat.getLen()); 743 } 744 745 private static void addHostToRack(Map<String, Set<String>> rackToNodes, 746 String rack, String host) { 747 Set<String> hosts = rackToNodes.get(rack); 748 if (hosts == null) { 749 hosts = new HashSet<String>(); 750 rackToNodes.put(rack, hosts); 751 } 752 hosts.add(host); 753 } 754 755 private Set<String> getHosts(Set<String> racks) { 756 Set<String> hosts = new HashSet<String>(); 757 for (String rack : racks) { 758 if (rackToNodes.containsKey(rack)) { 759 hosts.addAll(rackToNodes.get(rack)); 760 } 761 } 762 return hosts; 763 } 764 765 /** 766 * Accept a path only if any one of filters given in the 767 * constructor do. 768 */ 769 private static class MultiPathFilter implements PathFilter { 770 private List<PathFilter> filters; 771 772 public MultiPathFilter() { 773 this.filters = new ArrayList<PathFilter>(); 774 } 775 776 public MultiPathFilter(List<PathFilter> filters) { 777 this.filters = filters; 778 } 779 780 public void add(PathFilter one) { 781 filters.add(one); 782 } 783 784 public boolean accept(Path path) { 785 for (PathFilter filter : filters) { 786 if (filter.accept(path)) { 787 return true; 788 } 789 } 790 return false; 791 } 792 793 public String toString() { 794 StringBuffer buf = new StringBuffer(); 795 buf.append("["); 796 for (PathFilter f: filters) { 797 buf.append(f); 798 buf.append(","); 799 } 800 buf.append("]"); 801 return buf.toString(); 802 } 803 } 804 }