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.List;
024    
025    import org.apache.commons.logging.Log;
026    import org.apache.commons.logging.LogFactory;
027    import org.apache.hadoop.classification.InterfaceAudience;
028    import org.apache.hadoop.classification.InterfaceStability;
029    import org.apache.hadoop.conf.Configuration;
030    import org.apache.hadoop.fs.FileStatus;
031    import org.apache.hadoop.fs.FileSystem;
032    import org.apache.hadoop.fs.Path;
033    import org.apache.hadoop.fs.PathFilter;
034    import org.apache.hadoop.fs.BlockLocation;
035    import org.apache.hadoop.mapreduce.InputFormat;
036    import org.apache.hadoop.mapreduce.InputSplit;
037    import org.apache.hadoop.mapreduce.Job;
038    import org.apache.hadoop.mapreduce.JobContext;
039    import org.apache.hadoop.mapreduce.Mapper;
040    import org.apache.hadoop.mapreduce.security.TokenCache;
041    import org.apache.hadoop.util.ReflectionUtils;
042    import org.apache.hadoop.util.StringUtils;
043    
044    /** 
045     * A base class for file-based {@link InputFormat}s.
046     * 
047     * <p><code>FileInputFormat</code> is the base class for all file-based 
048     * <code>InputFormat</code>s. This provides a generic implementation of
049     * {@link #getSplits(JobContext)}.
050     * Subclasses of <code>FileInputFormat</code> can also override the 
051     * {@link #isSplitable(JobContext, Path)} method to ensure input-files are
052     * not split-up and are processed as a whole by {@link Mapper}s.
053     */
054    @InterfaceAudience.Public
055    @InterfaceStability.Stable
056    public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
057      public static final String INPUT_DIR = 
058        "mapreduce.input.fileinputformat.inputdir";
059      public static final String SPLIT_MAXSIZE = 
060        "mapreduce.input.fileinputformat.split.maxsize";
061      public static final String SPLIT_MINSIZE = 
062        "mapreduce.input.fileinputformat.split.minsize";
063      public static final String PATHFILTER_CLASS = 
064        "mapreduce.input.pathFilter.class";
065      public static final String NUM_INPUT_FILES =
066        "mapreduce.input.fileinputformat.numinputfiles";
067    
068      private static final Log LOG = LogFactory.getLog(FileInputFormat.class);
069    
070      private static final double SPLIT_SLOP = 1.1;   // 10% slop
071      
072      @Deprecated
073      public static enum Counter { 
074        BYTES_READ
075      }
076    
077      private static final PathFilter hiddenFileFilter = new PathFilter(){
078          public boolean accept(Path p){
079            String name = p.getName(); 
080            return !name.startsWith("_") && !name.startsWith("."); 
081          }
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       * Get the lower bound on split size imposed by the format.
108       * @return the number of bytes of the minimal split for this format
109       */
110      protected long getFormatMinSplitSize() {
111        return 1;
112      }
113    
114      /**
115       * Is the given filename splitable? Usually, true, but if the file is
116       * stream compressed, it will not be.
117       * 
118       * <code>FileInputFormat</code> implementations can override this and return
119       * <code>false</code> to ensure that individual input files are never split-up
120       * so that {@link Mapper}s process entire files.
121       * 
122       * @param context the job context
123       * @param filename the file name to check
124       * @return is this file splitable?
125       */
126      protected boolean isSplitable(JobContext context, Path filename) {
127        return true;
128      }
129    
130      /**
131       * Set a PathFilter to be applied to the input paths for the map-reduce job.
132       * @param job the job to modify
133       * @param filter the PathFilter class use for filtering the input paths.
134       */
135      public static void setInputPathFilter(Job job,
136                                            Class<? extends PathFilter> filter) {
137        job.getConfiguration().setClass(PATHFILTER_CLASS, filter, 
138                                        PathFilter.class);
139      }
140    
141      /**
142       * Set the minimum input split size
143       * @param job the job to modify
144       * @param size the minimum size
145       */
146      public static void setMinInputSplitSize(Job job,
147                                              long size) {
148        job.getConfiguration().setLong(SPLIT_MINSIZE, size);
149      }
150    
151      /**
152       * Get the minimum split size
153       * @param job the job
154       * @return the minimum number of bytes that can be in a split
155       */
156      public static long getMinSplitSize(JobContext job) {
157        return job.getConfiguration().getLong(SPLIT_MINSIZE, 1L);
158      }
159    
160      /**
161       * Set the maximum split size
162       * @param job the job to modify
163       * @param size the maximum split size
164       */
165      public static void setMaxInputSplitSize(Job job,
166                                              long size) {
167        job.getConfiguration().setLong(SPLIT_MAXSIZE, size);
168      }
169    
170      /**
171       * Get the maximum split size.
172       * @param context the job to look at.
173       * @return the maximum number of bytes a split can include
174       */
175      public static long getMaxSplitSize(JobContext context) {
176        return context.getConfiguration().getLong(SPLIT_MAXSIZE, 
177                                                  Long.MAX_VALUE);
178      }
179    
180      /**
181       * Get a PathFilter instance of the filter set for the input paths.
182       *
183       * @return the PathFilter instance set for the job, NULL if none has been set.
184       */
185      public static PathFilter getInputPathFilter(JobContext context) {
186        Configuration conf = context.getConfiguration();
187        Class<?> filterClass = conf.getClass(PATHFILTER_CLASS, null,
188            PathFilter.class);
189        return (filterClass != null) ?
190            (PathFilter) ReflectionUtils.newInstance(filterClass, conf) : null;
191      }
192    
193      /** List input directories.
194       * Subclasses may override to, e.g., select only files matching a regular
195       * expression. 
196       * 
197       * @param job the job to list input paths for
198       * @return array of FileStatus objects
199       * @throws IOException if zero items.
200       */
201      protected List<FileStatus> listStatus(JobContext job
202                                            ) throws IOException {
203        List<FileStatus> result = new ArrayList<FileStatus>();
204        Path[] dirs = getInputPaths(job);
205        if (dirs.length == 0) {
206          throw new IOException("No input paths specified in job");
207        }
208        
209        // get tokens for all the required FileSystems..
210        TokenCache.obtainTokensForNamenodes(job.getCredentials(), dirs, 
211                                            job.getConfiguration());
212    
213        List<IOException> errors = new ArrayList<IOException>();
214        
215        // creates a MultiPathFilter with the hiddenFileFilter and the
216        // user provided one (if any).
217        List<PathFilter> filters = new ArrayList<PathFilter>();
218        filters.add(hiddenFileFilter);
219        PathFilter jobFilter = getInputPathFilter(job);
220        if (jobFilter != null) {
221          filters.add(jobFilter);
222        }
223        PathFilter inputFilter = new MultiPathFilter(filters);
224        
225        for (int i=0; i < dirs.length; ++i) {
226          Path p = dirs[i];
227          FileSystem fs = p.getFileSystem(job.getConfiguration()); 
228          FileStatus[] matches = fs.globStatus(p, inputFilter);
229          if (matches == null) {
230            errors.add(new IOException("Input path does not exist: " + p));
231          } else if (matches.length == 0) {
232            errors.add(new IOException("Input Pattern " + p + " matches 0 files"));
233          } else {
234            for (FileStatus globStat: matches) {
235              if (globStat.isDirectory()) {
236                for(FileStatus stat: fs.listStatus(globStat.getPath(),
237                    inputFilter)) {
238                  result.add(stat);
239                }          
240              } else {
241                result.add(globStat);
242              }
243            }
244          }
245        }
246    
247        if (!errors.isEmpty()) {
248          throw new InvalidInputException(errors);
249        }
250        LOG.info("Total input paths to process : " + result.size()); 
251        return result;
252      }
253      
254      /**
255       * A factory that makes the split for this class. It can be overridden
256       * by sub-classes to make sub-types
257       */
258      protected FileSplit makeSplit(Path file, long start, long length, 
259                                    String[] hosts) {
260        return new FileSplit(file, start, length, hosts);
261      }
262    
263      /** 
264       * Generate the list of files and make them into FileSplits.
265       * @param job the job context
266       * @throws IOException
267       */
268      public List<InputSplit> getSplits(JobContext job) throws IOException {
269        long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
270        long maxSize = getMaxSplitSize(job);
271    
272        // generate splits
273        List<InputSplit> splits = new ArrayList<InputSplit>();
274        List<FileStatus> files = listStatus(job);
275        for (FileStatus file: files) {
276          Path path = file.getPath();
277          long length = file.getLen();
278          if (length != 0) {
279            FileSystem fs = path.getFileSystem(job.getConfiguration());
280            BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
281            if (isSplitable(job, path)) {
282              long blockSize = file.getBlockSize();
283              long splitSize = computeSplitSize(blockSize, minSize, maxSize);
284    
285              long bytesRemaining = length;
286              while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
287                int blkIndex = getBlockIndex(blkLocations, length-bytesRemaining);
288                splits.add(makeSplit(path, length-bytesRemaining, splitSize,
289                                         blkLocations[blkIndex].getHosts()));
290                bytesRemaining -= splitSize;
291              }
292    
293              if (bytesRemaining != 0) {
294                int blkIndex = getBlockIndex(blkLocations, length-bytesRemaining);
295                splits.add(makeSplit(path, length-bytesRemaining, bytesRemaining,
296                           blkLocations[blkIndex].getHosts()));
297              }
298            } else { // not splitable
299              splits.add(makeSplit(path, 0, length, blkLocations[0].getHosts()));
300            }
301          } else { 
302            //Create empty hosts array for zero length files
303            splits.add(makeSplit(path, 0, length, new String[0]));
304          }
305        }
306        // Save the number of input files for metrics/loadgen
307        job.getConfiguration().setLong(NUM_INPUT_FILES, files.size());
308        LOG.debug("Total # of splits: " + splits.size());
309        return splits;
310      }
311    
312      protected long computeSplitSize(long blockSize, long minSize,
313                                      long maxSize) {
314        return Math.max(minSize, Math.min(maxSize, blockSize));
315      }
316    
317      protected int getBlockIndex(BlockLocation[] blkLocations, 
318                                  long offset) {
319        for (int i = 0 ; i < blkLocations.length; i++) {
320          // is the offset inside this block?
321          if ((blkLocations[i].getOffset() <= offset) &&
322              (offset < blkLocations[i].getOffset() + blkLocations[i].getLength())){
323            return i;
324          }
325        }
326        BlockLocation last = blkLocations[blkLocations.length -1];
327        long fileLength = last.getOffset() + last.getLength() -1;
328        throw new IllegalArgumentException("Offset " + offset + 
329                                           " is outside of file (0.." +
330                                           fileLength + ")");
331      }
332    
333      /**
334       * Sets the given comma separated paths as the list of inputs 
335       * for the map-reduce job.
336       * 
337       * @param job the job
338       * @param commaSeparatedPaths Comma separated paths to be set as 
339       *        the list of inputs for the map-reduce job.
340       */
341      public static void setInputPaths(Job job, 
342                                       String commaSeparatedPaths
343                                       ) throws IOException {
344        setInputPaths(job, StringUtils.stringToPath(
345                            getPathStrings(commaSeparatedPaths)));
346      }
347    
348      /**
349       * Add the given comma separated paths to the list of inputs for
350       *  the map-reduce job.
351       * 
352       * @param job The job to modify
353       * @param commaSeparatedPaths Comma separated paths to be added to
354       *        the list of inputs for the map-reduce job.
355       */
356      public static void addInputPaths(Job job, 
357                                       String commaSeparatedPaths
358                                       ) throws IOException {
359        for (String str : getPathStrings(commaSeparatedPaths)) {
360          addInputPath(job, new Path(str));
361        }
362      }
363    
364      /**
365       * Set the array of {@link Path}s as the list of inputs
366       * for the map-reduce job.
367       * 
368       * @param job The job to modify 
369       * @param inputPaths the {@link Path}s of the input directories/files 
370       * for the map-reduce job.
371       */ 
372      public static void setInputPaths(Job job, 
373                                       Path... inputPaths) throws IOException {
374        Configuration conf = job.getConfiguration();
375        Path path = inputPaths[0].getFileSystem(conf).makeQualified(inputPaths[0]);
376        StringBuffer str = new StringBuffer(StringUtils.escapeString(path.toString()));
377        for(int i = 1; i < inputPaths.length;i++) {
378          str.append(StringUtils.COMMA_STR);
379          path = inputPaths[i].getFileSystem(conf).makeQualified(inputPaths[i]);
380          str.append(StringUtils.escapeString(path.toString()));
381        }
382        conf.set(INPUT_DIR, str.toString());
383      }
384    
385      /**
386       * Add a {@link Path} to the list of inputs for the map-reduce job.
387       * 
388       * @param job The {@link Job} to modify
389       * @param path {@link Path} to be added to the list of inputs for 
390       *            the map-reduce job.
391       */
392      public static void addInputPath(Job job, 
393                                      Path path) throws IOException {
394        Configuration conf = job.getConfiguration();
395        path = path.getFileSystem(conf).makeQualified(path);
396        String dirStr = StringUtils.escapeString(path.toString());
397        String dirs = conf.get(INPUT_DIR);
398        conf.set(INPUT_DIR, dirs == null ? dirStr : dirs + "," + dirStr);
399      }
400      
401      // This method escapes commas in the glob pattern of the given paths.
402      private static String[] getPathStrings(String commaSeparatedPaths) {
403        int length = commaSeparatedPaths.length();
404        int curlyOpen = 0;
405        int pathStart = 0;
406        boolean globPattern = false;
407        List<String> pathStrings = new ArrayList<String>();
408        
409        for (int i=0; i<length; i++) {
410          char ch = commaSeparatedPaths.charAt(i);
411          switch(ch) {
412            case '{' : {
413              curlyOpen++;
414              if (!globPattern) {
415                globPattern = true;
416              }
417              break;
418            }
419            case '}' : {
420              curlyOpen--;
421              if (curlyOpen == 0 && globPattern) {
422                globPattern = false;
423              }
424              break;
425            }
426            case ',' : {
427              if (!globPattern) {
428                pathStrings.add(commaSeparatedPaths.substring(pathStart, i));
429                pathStart = i + 1 ;
430              }
431              break;
432            }
433            default:
434              continue; // nothing special to do for this character
435          }
436        }
437        pathStrings.add(commaSeparatedPaths.substring(pathStart, length));
438        
439        return pathStrings.toArray(new String[0]);
440      }
441      
442      /**
443       * Get the list of input {@link Path}s for the map-reduce job.
444       * 
445       * @param context The job
446       * @return the list of input {@link Path}s for the map-reduce job.
447       */
448      public static Path[] getInputPaths(JobContext context) {
449        String dirs = context.getConfiguration().get(INPUT_DIR, "");
450        String [] list = StringUtils.split(dirs);
451        Path[] result = new Path[list.length];
452        for (int i = 0; i < list.length; i++) {
453          result[i] = new Path(StringUtils.unEscapeString(list[i]));
454        }
455        return result;
456      }
457    
458    }