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
019package org.apache.hadoop.mapred;
020
021import java.io.IOException;
022import java.util.HashSet;
023import java.util.Set;
024
025import org.apache.hadoop.classification.InterfaceAudience;
026import org.apache.hadoop.classification.InterfaceStability;
027import org.apache.hadoop.fs.BlockLocation;
028import org.apache.hadoop.fs.FileStatus;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.mapred.lib.CombineFileSplit;
032
033/**
034 * A sub-collection of input files. Unlike {@link FileSplit}, MultiFileSplit 
035 * class does not represent a split of a file, but a split of input files 
036 * into smaller sets. The atomic unit of split is a file. <br> 
037 * MultiFileSplit can be used to implement {@link RecordReader}'s, with 
038 * reading one record per file.
039 * @see FileSplit
040 * @see MultiFileInputFormat 
041 */
042@InterfaceAudience.Public
043@InterfaceStability.Stable
044public class MultiFileSplit extends CombineFileSplit {
045
046  MultiFileSplit() {}
047  
048  public MultiFileSplit(JobConf job, Path[] files, long[] lengths) {
049    super(job, files, lengths);
050  }
051
052  public String[] getLocations() throws IOException {
053    HashSet<String> hostSet = new HashSet<String>();
054    for (Path file : getPaths()) {
055      FileSystem fs = file.getFileSystem(getJob());
056      FileStatus status = fs.getFileStatus(file);
057      BlockLocation[] blkLocations = fs.getFileBlockLocations(status,
058                                          0, status.getLen());
059      if (blkLocations != null && blkLocations.length > 0) {
060        addToSet(hostSet, blkLocations[0].getHosts());
061      }
062    }
063    return hostSet.toArray(new String[hostSet.size()]);
064  }
065
066  private void addToSet(Set<String> set, String[] array) {
067    for(String s:array)
068      set.add(s); 
069  }
070
071  @Override
072  public String toString() {
073    StringBuffer sb = new StringBuffer();
074    for(int i=0; i < getPaths().length; i++) {
075      sb.append(getPath(i).toUri().getPath() + ":0+" + getLength(i));
076      if (i < getPaths().length -1) {
077        sb.append("\n");
078      }
079    }
080
081    return sb.toString();
082  }
083}
084