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.List; 024 025 import org.apache.hadoop.classification.InterfaceAudience; 026 import org.apache.hadoop.classification.InterfaceStability; 027 import org.apache.hadoop.fs.FileSystem; 028 import org.apache.hadoop.fs.FileUtil; 029 import org.apache.hadoop.fs.Path; 030 031 /** 032 * An abstract {@link InputFormat} that returns {@link MultiFileSplit}'s 033 * in {@link #getSplits(JobConf, int)} method. Splits are constructed from 034 * the files under the input paths. Each split returned contains <i>nearly</i> 035 * equal content length. <br> 036 * Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} 037 * to construct <code>RecordReader</code>'s for <code>MultiFileSplit</code>'s. 038 * @see MultiFileSplit 039 * @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileInputFormat} instead 040 */ 041 @Deprecated 042 @InterfaceAudience.Public 043 @InterfaceStability.Stable 044 public abstract class MultiFileInputFormat<K, V> 045 extends FileInputFormat<K, V> { 046 047 @Override 048 public InputSplit[] getSplits(JobConf job, int numSplits) 049 throws IOException { 050 051 Path[] paths = FileUtil.stat2Paths(listStatus(job)); 052 List<MultiFileSplit> splits = new ArrayList<MultiFileSplit>(Math.min(numSplits, paths.length)); 053 if (paths.length != 0) { 054 // HADOOP-1818: Manage splits only if there are paths 055 long[] lengths = new long[paths.length]; 056 long totLength = 0; 057 for(int i=0; i<paths.length; i++) { 058 FileSystem fs = paths[i].getFileSystem(job); 059 lengths[i] = fs.getContentSummary(paths[i]).getLength(); 060 totLength += lengths[i]; 061 } 062 double avgLengthPerSplit = ((double)totLength) / numSplits; 063 long cumulativeLength = 0; 064 065 int startIndex = 0; 066 067 for(int i=0; i<numSplits; i++) { 068 int splitSize = findSize(i, avgLengthPerSplit, cumulativeLength 069 , startIndex, lengths); 070 if (splitSize != 0) { 071 // HADOOP-1818: Manage split only if split size is not equals to 0 072 Path[] splitPaths = new Path[splitSize]; 073 long[] splitLengths = new long[splitSize]; 074 System.arraycopy(paths, startIndex, splitPaths , 0, splitSize); 075 System.arraycopy(lengths, startIndex, splitLengths , 0, splitSize); 076 splits.add(new MultiFileSplit(job, splitPaths, splitLengths)); 077 startIndex += splitSize; 078 for(long l: splitLengths) { 079 cumulativeLength += l; 080 } 081 } 082 } 083 } 084 return splits.toArray(new MultiFileSplit[splits.size()]); 085 } 086 087 private int findSize(int splitIndex, double avgLengthPerSplit 088 , long cumulativeLength , int startIndex, long[] lengths) { 089 090 if(splitIndex == lengths.length - 1) 091 return lengths.length - startIndex; 092 093 long goalLength = (long)((splitIndex + 1) * avgLengthPerSplit); 094 long partialLength = 0; 095 // accumulate till just above the goal length; 096 for(int i = startIndex; i < lengths.length; i++) { 097 partialLength += lengths[i]; 098 if(partialLength + cumulativeLength >= goalLength) { 099 return i - startIndex + 1; 100 } 101 } 102 return lengths.length - startIndex; 103 } 104 105 @Override 106 public abstract RecordReader<K, V> getRecordReader(InputSplit split, 107 JobConf job, Reporter reporter) 108 throws IOException; 109 }