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