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.mapred; 020 021 import java.io.IOException; 022 import java.util.ArrayList; 023 import java.util.Collections; 024 import java.util.Comparator; 025 import java.util.HashSet; 026 import java.util.IdentityHashMap; 027 import java.util.LinkedList; 028 import java.util.List; 029 import java.util.Map; 030 import java.util.Set; 031 032 import org.apache.commons.logging.Log; 033 import org.apache.commons.logging.LogFactory; 034 import org.apache.hadoop.classification.InterfaceAudience; 035 import org.apache.hadoop.classification.InterfaceStability; 036 import org.apache.hadoop.fs.BlockLocation; 037 import org.apache.hadoop.fs.FileStatus; 038 import org.apache.hadoop.fs.FileSystem; 039 import org.apache.hadoop.fs.Path; 040 import org.apache.hadoop.fs.PathFilter; 041 import org.apache.hadoop.mapreduce.security.TokenCache; 042 import org.apache.hadoop.net.NetworkTopology; 043 import org.apache.hadoop.net.Node; 044 import org.apache.hadoop.net.NodeBase; 045 import org.apache.hadoop.util.ReflectionUtils; 046 import org.apache.hadoop.util.StringUtils; 047 048 /** 049 * A base class for file-based {@link InputFormat}. 050 * 051 * <p><code>FileInputFormat</code> is the base class for all file-based 052 * <code>InputFormat</code>s. This provides a generic implementation of 053 * {@link #getSplits(JobConf, int)}. 054 * Subclasses of <code>FileInputFormat</code> can also override the 055 * {@link #isSplitable(FileSystem, Path)} method to ensure input-files are 056 * not split-up and are processed as a whole by {@link Mapper}s. 057 */ 058 @InterfaceAudience.Public 059 @InterfaceStability.Stable 060 public abstract class FileInputFormat<K, V> implements InputFormat<K, V> { 061 062 public static final Log LOG = 063 LogFactory.getLog(FileInputFormat.class); 064 065 @Deprecated 066 public static enum Counter { 067 BYTES_READ 068 } 069 070 public static final String NUM_INPUT_FILES = 071 org.apache.hadoop.mapreduce.lib.input.FileInputFormat.NUM_INPUT_FILES; 072 073 private static final double SPLIT_SLOP = 1.1; // 10% slop 074 075 private long minSplitSize = 1; 076 private static final PathFilter hiddenFileFilter = new PathFilter(){ 077 public boolean accept(Path p){ 078 String name = p.getName(); 079 return !name.startsWith("_") && !name.startsWith("."); 080 } 081 }; 082 protected void setMinSplitSize(long minSplitSize) { 083 this.minSplitSize = minSplitSize; 084 } 085 086 /** 087 * Proxy PathFilter that accepts a path only if all filters given in the 088 * constructor do. Used by the listPaths() to apply the built-in 089 * hiddenFileFilter together with a user provided one (if any). 090 */ 091 private static class MultiPathFilter implements PathFilter { 092 private List<PathFilter> filters; 093 094 public MultiPathFilter(List<PathFilter> filters) { 095 this.filters = filters; 096 } 097 098 public boolean accept(Path path) { 099 for (PathFilter filter : filters) { 100 if (!filter.accept(path)) { 101 return false; 102 } 103 } 104 return true; 105 } 106 } 107 108 /** 109 * Is the given filename splitable? Usually, true, but if the file is 110 * stream compressed, it will not be. 111 * 112 * <code>FileInputFormat</code> implementations can override this and return 113 * <code>false</code> to ensure that individual input files are never split-up 114 * so that {@link Mapper}s process entire files. 115 * 116 * @param fs the file system that the file is on 117 * @param filename the file name to check 118 * @return is this file splitable? 119 */ 120 protected boolean isSplitable(FileSystem fs, Path filename) { 121 return true; 122 } 123 124 public abstract RecordReader<K, V> getRecordReader(InputSplit split, 125 JobConf job, 126 Reporter reporter) 127 throws IOException; 128 129 /** 130 * Set a PathFilter to be applied to the input paths for the map-reduce job. 131 * 132 * @param filter the PathFilter class use for filtering the input paths. 133 */ 134 public static void setInputPathFilter(JobConf conf, 135 Class<? extends PathFilter> filter) { 136 conf.setClass(org.apache.hadoop.mapreduce.lib.input. 137 FileInputFormat.PATHFILTER_CLASS, filter, PathFilter.class); 138 } 139 140 /** 141 * Get a PathFilter instance of the filter set for the input paths. 142 * 143 * @return the PathFilter instance set for the job, NULL if none has been set. 144 */ 145 public static PathFilter getInputPathFilter(JobConf conf) { 146 Class<? extends PathFilter> filterClass = conf.getClass( 147 org.apache.hadoop.mapreduce.lib.input.FileInputFormat.PATHFILTER_CLASS, 148 null, PathFilter.class); 149 return (filterClass != null) ? 150 ReflectionUtils.newInstance(filterClass, conf) : null; 151 } 152 153 /** 154 * Add files in the input path recursively into the results. 155 * @param result 156 * The List to store all files. 157 * @param fs 158 * The FileSystem. 159 * @param path 160 * The input path. 161 * @param inputFilter 162 * The input filter that can be used to filter files/dirs. 163 * @throws IOException 164 */ 165 protected void addInputPathRecursively(List<FileStatus> result, 166 FileSystem fs, Path path, PathFilter inputFilter) 167 throws IOException { 168 for(FileStatus stat: fs.listStatus(path, inputFilter)) { 169 if (stat.isDirectory()) { 170 addInputPathRecursively(result, fs, stat.getPath(), inputFilter); 171 } else { 172 result.add(stat); 173 } 174 } 175 } 176 177 /** List input directories. 178 * Subclasses may override to, e.g., select only files matching a regular 179 * expression. 180 * 181 * @param job the job to list input paths for 182 * @return array of FileStatus objects 183 * @throws IOException if zero items. 184 */ 185 protected FileStatus[] listStatus(JobConf job) throws IOException { 186 Path[] dirs = getInputPaths(job); 187 if (dirs.length == 0) { 188 throw new IOException("No input paths specified in job"); 189 } 190 191 // get tokens for all the required FileSystems.. 192 TokenCache.obtainTokensForNamenodes(job.getCredentials(), dirs, job); 193 194 // Whether we need to recursive look into the directory structure 195 boolean recursive = job.getBoolean("mapred.input.dir.recursive", false); 196 197 List<FileStatus> result = new ArrayList<FileStatus>(); 198 List<IOException> errors = new ArrayList<IOException>(); 199 200 // creates a MultiPathFilter with the hiddenFileFilter and the 201 // user provided one (if any). 202 List<PathFilter> filters = new ArrayList<PathFilter>(); 203 filters.add(hiddenFileFilter); 204 PathFilter jobFilter = getInputPathFilter(job); 205 if (jobFilter != null) { 206 filters.add(jobFilter); 207 } 208 PathFilter inputFilter = new MultiPathFilter(filters); 209 210 for (Path p: dirs) { 211 FileSystem fs = p.getFileSystem(job); 212 FileStatus[] matches = fs.globStatus(p, inputFilter); 213 if (matches == null) { 214 errors.add(new IOException("Input path does not exist: " + p)); 215 } else if (matches.length == 0) { 216 errors.add(new IOException("Input Pattern " + p + " matches 0 files")); 217 } else { 218 for (FileStatus globStat: matches) { 219 if (globStat.isDirectory()) { 220 for(FileStatus stat: fs.listStatus(globStat.getPath(), 221 inputFilter)) { 222 if (recursive && stat.isDirectory()) { 223 addInputPathRecursively(result, fs, stat.getPath(), inputFilter); 224 } else { 225 result.add(stat); 226 } 227 } 228 } else { 229 result.add(globStat); 230 } 231 } 232 } 233 } 234 235 if (!errors.isEmpty()) { 236 throw new InvalidInputException(errors); 237 } 238 LOG.info("Total input paths to process : " + result.size()); 239 return result.toArray(new FileStatus[result.size()]); 240 } 241 242 /** 243 * A factory that makes the split for this class. It can be overridden 244 * by sub-classes to make sub-types 245 */ 246 protected FileSplit makeSplit(Path file, long start, long length, 247 String[] hosts) { 248 return new FileSplit(file, start, length, hosts); 249 } 250 251 /** Splits files returned by {@link #listStatus(JobConf)} when 252 * they're too big.*/ 253 @SuppressWarnings("deprecation") 254 public InputSplit[] getSplits(JobConf job, int numSplits) 255 throws IOException { 256 FileStatus[] files = listStatus(job); 257 258 // Save the number of input files for metrics/loadgen 259 job.setLong(NUM_INPUT_FILES, files.length); 260 long totalSize = 0; // compute total size 261 for (FileStatus file: files) { // check we have valid files 262 if (file.isDirectory()) { 263 throw new IOException("Not a file: "+ file.getPath()); 264 } 265 totalSize += file.getLen(); 266 } 267 268 long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits); 269 long minSize = Math.max(job.getLong(org.apache.hadoop.mapreduce.lib.input. 270 FileInputFormat.SPLIT_MINSIZE, 1), minSplitSize); 271 272 // generate splits 273 ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits); 274 NetworkTopology clusterMap = new NetworkTopology(); 275 for (FileStatus file: files) { 276 Path path = file.getPath(); 277 FileSystem fs = path.getFileSystem(job); 278 long length = file.getLen(); 279 BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length); 280 if ((length != 0) && isSplitable(fs, path)) { 281 long blockSize = file.getBlockSize(); 282 long splitSize = computeSplitSize(goalSize, minSize, blockSize); 283 284 long bytesRemaining = length; 285 while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) { 286 String[] splitHosts = getSplitHosts(blkLocations, 287 length-bytesRemaining, splitSize, clusterMap); 288 splits.add(makeSplit(path, length-bytesRemaining, splitSize, 289 splitHosts)); 290 bytesRemaining -= splitSize; 291 } 292 293 if (bytesRemaining != 0) { 294 String[] splitHosts = getSplitHosts(blkLocations, length 295 - bytesRemaining, bytesRemaining, clusterMap); 296 splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining, 297 splitHosts)); 298 } 299 } else if (length != 0) { 300 String[] splitHosts = getSplitHosts(blkLocations,0,length,clusterMap); 301 splits.add(makeSplit(path, 0, length, splitHosts)); 302 } else { 303 //Create empty hosts array for zero length files 304 splits.add(makeSplit(path, 0, length, new String[0])); 305 } 306 } 307 LOG.debug("Total # of splits: " + splits.size()); 308 return splits.toArray(new FileSplit[splits.size()]); 309 } 310 311 protected long computeSplitSize(long goalSize, long minSize, 312 long blockSize) { 313 return Math.max(minSize, Math.min(goalSize, blockSize)); 314 } 315 316 protected int getBlockIndex(BlockLocation[] blkLocations, 317 long offset) { 318 for (int i = 0 ; i < blkLocations.length; i++) { 319 // is the offset inside this block? 320 if ((blkLocations[i].getOffset() <= offset) && 321 (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())){ 322 return i; 323 } 324 } 325 BlockLocation last = blkLocations[blkLocations.length -1]; 326 long fileLength = last.getOffset() + last.getLength() -1; 327 throw new IllegalArgumentException("Offset " + offset + 328 " is outside of file (0.." + 329 fileLength + ")"); 330 } 331 332 /** 333 * Sets the given comma separated paths as the list of inputs 334 * for the map-reduce job. 335 * 336 * @param conf Configuration of the job 337 * @param commaSeparatedPaths Comma separated paths to be set as 338 * the list of inputs for the map-reduce job. 339 */ 340 public static void setInputPaths(JobConf conf, String commaSeparatedPaths) { 341 setInputPaths(conf, StringUtils.stringToPath( 342 getPathStrings(commaSeparatedPaths))); 343 } 344 345 /** 346 * Add the given comma separated paths to the list of inputs for 347 * the map-reduce job. 348 * 349 * @param conf The configuration of the job 350 * @param commaSeparatedPaths Comma separated paths to be added to 351 * the list of inputs for the map-reduce job. 352 */ 353 public static void addInputPaths(JobConf conf, String commaSeparatedPaths) { 354 for (String str : getPathStrings(commaSeparatedPaths)) { 355 addInputPath(conf, new Path(str)); 356 } 357 } 358 359 /** 360 * Set the array of {@link Path}s as the list of inputs 361 * for the map-reduce job. 362 * 363 * @param conf Configuration of the job. 364 * @param inputPaths the {@link Path}s of the input directories/files 365 * for the map-reduce job. 366 */ 367 public static void setInputPaths(JobConf conf, Path... inputPaths) { 368 Path path = new Path(conf.getWorkingDirectory(), inputPaths[0]); 369 StringBuffer str = new StringBuffer(StringUtils.escapeString(path.toString())); 370 for(int i = 1; i < inputPaths.length;i++) { 371 str.append(StringUtils.COMMA_STR); 372 path = new Path(conf.getWorkingDirectory(), inputPaths[i]); 373 str.append(StringUtils.escapeString(path.toString())); 374 } 375 conf.set(org.apache.hadoop.mapreduce.lib.input. 376 FileInputFormat.INPUT_DIR, str.toString()); 377 } 378 379 /** 380 * Add a {@link Path} to the list of inputs for the map-reduce job. 381 * 382 * @param conf The configuration of the job 383 * @param path {@link Path} to be added to the list of inputs for 384 * the map-reduce job. 385 */ 386 public static void addInputPath(JobConf conf, Path path ) { 387 path = new Path(conf.getWorkingDirectory(), path); 388 String dirStr = StringUtils.escapeString(path.toString()); 389 String dirs = conf.get(org.apache.hadoop.mapreduce.lib.input. 390 FileInputFormat.INPUT_DIR); 391 conf.set(org.apache.hadoop.mapreduce.lib.input. 392 FileInputFormat.INPUT_DIR, dirs == null ? dirStr : 393 dirs + StringUtils.COMMA_STR + dirStr); 394 } 395 396 // This method escapes commas in the glob pattern of the given paths. 397 private static String[] getPathStrings(String commaSeparatedPaths) { 398 int length = commaSeparatedPaths.length(); 399 int curlyOpen = 0; 400 int pathStart = 0; 401 boolean globPattern = false; 402 List<String> pathStrings = new ArrayList<String>(); 403 404 for (int i=0; i<length; i++) { 405 char ch = commaSeparatedPaths.charAt(i); 406 switch(ch) { 407 case '{' : { 408 curlyOpen++; 409 if (!globPattern) { 410 globPattern = true; 411 } 412 break; 413 } 414 case '}' : { 415 curlyOpen--; 416 if (curlyOpen == 0 && globPattern) { 417 globPattern = false; 418 } 419 break; 420 } 421 case ',' : { 422 if (!globPattern) { 423 pathStrings.add(commaSeparatedPaths.substring(pathStart, i)); 424 pathStart = i + 1 ; 425 } 426 break; 427 } 428 default: 429 continue; // nothing special to do for this character 430 } 431 } 432 pathStrings.add(commaSeparatedPaths.substring(pathStart, length)); 433 434 return pathStrings.toArray(new String[0]); 435 } 436 437 /** 438 * Get the list of input {@link Path}s for the map-reduce job. 439 * 440 * @param conf The configuration of the job 441 * @return the list of input {@link Path}s for the map-reduce job. 442 */ 443 public static Path[] getInputPaths(JobConf conf) { 444 String dirs = conf.get(org.apache.hadoop.mapreduce.lib.input. 445 FileInputFormat.INPUT_DIR, ""); 446 String [] list = StringUtils.split(dirs); 447 Path[] result = new Path[list.length]; 448 for (int i = 0; i < list.length; i++) { 449 result[i] = new Path(StringUtils.unEscapeString(list[i])); 450 } 451 return result; 452 } 453 454 455 private void sortInDescendingOrder(List<NodeInfo> mylist) { 456 Collections.sort(mylist, new Comparator<NodeInfo> () { 457 public int compare(NodeInfo obj1, NodeInfo obj2) { 458 459 if (obj1 == null || obj2 == null) 460 return -1; 461 462 if (obj1.getValue() == obj2.getValue()) { 463 return 0; 464 } 465 else { 466 return ((obj1.getValue() < obj2.getValue()) ? 1 : -1); 467 } 468 } 469 } 470 ); 471 } 472 473 /** 474 * This function identifies and returns the hosts that contribute 475 * most for a given split. For calculating the contribution, rack 476 * locality is treated on par with host locality, so hosts from racks 477 * that contribute the most are preferred over hosts on racks that 478 * contribute less 479 * @param blkLocations The list of block locations 480 * @param offset 481 * @param splitSize 482 * @return array of hosts that contribute most to this split 483 * @throws IOException 484 */ 485 protected String[] getSplitHosts(BlockLocation[] blkLocations, 486 long offset, long splitSize, NetworkTopology clusterMap) 487 throws IOException { 488 489 int startIndex = getBlockIndex(blkLocations, offset); 490 491 long bytesInThisBlock = blkLocations[startIndex].getOffset() + 492 blkLocations[startIndex].getLength() - offset; 493 494 //If this is the only block, just return 495 if (bytesInThisBlock >= splitSize) { 496 return blkLocations[startIndex].getHosts(); 497 } 498 499 long bytesInFirstBlock = bytesInThisBlock; 500 int index = startIndex + 1; 501 splitSize -= bytesInThisBlock; 502 503 while (splitSize > 0) { 504 bytesInThisBlock = 505 Math.min(splitSize, blkLocations[index++].getLength()); 506 splitSize -= bytesInThisBlock; 507 } 508 509 long bytesInLastBlock = bytesInThisBlock; 510 int endIndex = index - 1; 511 512 Map <Node,NodeInfo> hostsMap = new IdentityHashMap<Node,NodeInfo>(); 513 Map <Node,NodeInfo> racksMap = new IdentityHashMap<Node,NodeInfo>(); 514 String [] allTopos = new String[0]; 515 516 // Build the hierarchy and aggregate the contribution of 517 // bytes at each level. See TestGetSplitHosts.java 518 519 for (index = startIndex; index <= endIndex; index++) { 520 521 // Establish the bytes in this block 522 if (index == startIndex) { 523 bytesInThisBlock = bytesInFirstBlock; 524 } 525 else if (index == endIndex) { 526 bytesInThisBlock = bytesInLastBlock; 527 } 528 else { 529 bytesInThisBlock = blkLocations[index].getLength(); 530 } 531 532 allTopos = blkLocations[index].getTopologyPaths(); 533 534 // If no topology information is available, just 535 // prefix a fakeRack 536 if (allTopos.length == 0) { 537 allTopos = fakeRacks(blkLocations, index); 538 } 539 540 // NOTE: This code currently works only for one level of 541 // hierarchy (rack/host). However, it is relatively easy 542 // to extend this to support aggregation at different 543 // levels 544 545 for (String topo: allTopos) { 546 547 Node node, parentNode; 548 NodeInfo nodeInfo, parentNodeInfo; 549 550 node = clusterMap.getNode(topo); 551 552 if (node == null) { 553 node = new NodeBase(topo); 554 clusterMap.add(node); 555 } 556 557 nodeInfo = hostsMap.get(node); 558 559 if (nodeInfo == null) { 560 nodeInfo = new NodeInfo(node); 561 hostsMap.put(node,nodeInfo); 562 parentNode = node.getParent(); 563 parentNodeInfo = racksMap.get(parentNode); 564 if (parentNodeInfo == null) { 565 parentNodeInfo = new NodeInfo(parentNode); 566 racksMap.put(parentNode,parentNodeInfo); 567 } 568 parentNodeInfo.addLeaf(nodeInfo); 569 } 570 else { 571 nodeInfo = hostsMap.get(node); 572 parentNode = node.getParent(); 573 parentNodeInfo = racksMap.get(parentNode); 574 } 575 576 nodeInfo.addValue(index, bytesInThisBlock); 577 parentNodeInfo.addValue(index, bytesInThisBlock); 578 579 } // for all topos 580 581 } // for all indices 582 583 return identifyHosts(allTopos.length, racksMap); 584 } 585 586 private String[] identifyHosts(int replicationFactor, 587 Map<Node,NodeInfo> racksMap) { 588 589 String [] retVal = new String[replicationFactor]; 590 591 List <NodeInfo> rackList = new LinkedList<NodeInfo>(); 592 593 rackList.addAll(racksMap.values()); 594 595 // Sort the racks based on their contribution to this split 596 sortInDescendingOrder(rackList); 597 598 boolean done = false; 599 int index = 0; 600 601 // Get the host list for all our aggregated items, sort 602 // them and return the top entries 603 for (NodeInfo ni: rackList) { 604 605 Set<NodeInfo> hostSet = ni.getLeaves(); 606 607 List<NodeInfo>hostList = new LinkedList<NodeInfo>(); 608 hostList.addAll(hostSet); 609 610 // Sort the hosts in this rack based on their contribution 611 sortInDescendingOrder(hostList); 612 613 for (NodeInfo host: hostList) { 614 // Strip out the port number from the host name 615 retVal[index++] = host.node.getName().split(":")[0]; 616 if (index == replicationFactor) { 617 done = true; 618 break; 619 } 620 } 621 622 if (done == true) { 623 break; 624 } 625 } 626 return retVal; 627 } 628 629 private String[] fakeRacks(BlockLocation[] blkLocations, int index) 630 throws IOException { 631 String[] allHosts = blkLocations[index].getHosts(); 632 String[] allTopos = new String[allHosts.length]; 633 for (int i = 0; i < allHosts.length; i++) { 634 allTopos[i] = NetworkTopology.DEFAULT_RACK + "/" + allHosts[i]; 635 } 636 return allTopos; 637 } 638 639 640 private static class NodeInfo { 641 final Node node; 642 final Set<Integer> blockIds; 643 final Set<NodeInfo> leaves; 644 645 private long value; 646 647 NodeInfo(Node node) { 648 this.node = node; 649 blockIds = new HashSet<Integer>(); 650 leaves = new HashSet<NodeInfo>(); 651 } 652 653 long getValue() {return value;} 654 655 void addValue(int blockIndex, long value) { 656 if (blockIds.add(blockIndex) == true) { 657 this.value += value; 658 } 659 } 660 661 Set<NodeInfo> getLeaves() { return leaves;} 662 663 void addLeaf(NodeInfo nodeInfo) { 664 leaves.add(nodeInfo); 665 } 666 } 667 }