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.lib;
020    
021    import java.io.IOException;
022    import java.util.Iterator;
023    import java.util.TreeMap;
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.Path;
029    import org.apache.hadoop.mapred.JobConf;
030    import org.apache.hadoop.mapred.FileOutputFormat;
031    import org.apache.hadoop.mapred.RecordWriter;
032    import org.apache.hadoop.mapred.Reporter;
033    import org.apache.hadoop.mapreduce.JobContext;
034    import org.apache.hadoop.mapreduce.MRJobConfig;
035    import org.apache.hadoop.util.Progressable;
036    
037    /**
038     * This abstract class extends the FileOutputFormat, allowing to write the
039     * output data to different output files. There are three basic use cases for
040     * this class.
041     * 
042     * Case one: This class is used for a map reduce job with at least one reducer.
043     * The reducer wants to write data to different files depending on the actual
044     * keys. It is assumed that a key (or value) encodes the actual key (value)
045     * and the desired location for the actual key (value).
046     * 
047     * Case two: This class is used for a map only job. The job wants to use an
048     * output file name that is either a part of the input file name of the input
049     * data, or some derivation of it.
050     * 
051     * Case three: This class is used for a map only job. The job wants to use an
052     * output file name that depends on both the keys and the input file name,
053     * 
054     * @deprecated Use 
055     * {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead
056     */
057    @Deprecated
058    @InterfaceAudience.Public
059    @InterfaceStability.Stable
060    public abstract class MultipleOutputFormat<K, V>
061    extends FileOutputFormat<K, V> {
062    
063      /**
064       * Create a composite record writer that can write key/value data to different
065       * output files
066       * 
067       * @param fs
068       *          the file system to use
069       * @param job
070       *          the job conf for the job
071       * @param name
072       *          the leaf file name for the output file (such as part-00000")
073       * @param arg3
074       *          a progressable for reporting progress.
075       * @return a composite record writer
076       * @throws IOException
077       */
078      public RecordWriter<K, V> getRecordWriter(FileSystem fs, JobConf job,
079          String name, Progressable arg3) throws IOException {
080    
081        final FileSystem myFS = fs;
082        final String myName = generateLeafFileName(name);
083        final JobConf myJob = job;
084        final Progressable myProgressable = arg3;
085    
086        return new RecordWriter<K, V>() {
087    
088          // a cache storing the record writers for different output files.
089          TreeMap<String, RecordWriter<K, V>> recordWriters = new TreeMap<String, RecordWriter<K, V>>();
090    
091          public void write(K key, V value) throws IOException {
092    
093            // get the file name based on the key
094            String keyBasedPath = generateFileNameForKeyValue(key, value, myName);
095    
096            // get the file name based on the input file name
097            String finalPath = getInputFileBasedOutputFileName(myJob, keyBasedPath);
098    
099            // get the actual key
100            K actualKey = generateActualKey(key, value);
101            V actualValue = generateActualValue(key, value);
102    
103            RecordWriter<K, V> rw = this.recordWriters.get(finalPath);
104            if (rw == null) {
105              // if we don't have the record writer yet for the final path, create
106              // one
107              // and add it to the cache
108              rw = getBaseRecordWriter(myFS, myJob, finalPath, myProgressable);
109              this.recordWriters.put(finalPath, rw);
110            }
111            rw.write(actualKey, actualValue);
112          };
113    
114          public void close(Reporter reporter) throws IOException {
115            Iterator<String> keys = this.recordWriters.keySet().iterator();
116            while (keys.hasNext()) {
117              RecordWriter<K, V> rw = this.recordWriters.get(keys.next());
118              rw.close(reporter);
119            }
120            this.recordWriters.clear();
121          };
122        };
123      }
124    
125      /**
126       * Generate the leaf name for the output file name. The default behavior does
127       * not change the leaf file name (such as part-00000)
128       * 
129       * @param name
130       *          the leaf file name for the output file
131       * @return the given leaf file name
132       */
133      protected String generateLeafFileName(String name) {
134        return name;
135      }
136    
137      /**
138       * Generate the file output file name based on the given key and the leaf file
139       * name. The default behavior is that the file name does not depend on the
140       * key.
141       * 
142       * @param key
143       *          the key of the output data
144       * @param name
145       *          the leaf file name
146       * @return generated file name
147       */
148      protected String generateFileNameForKeyValue(K key, V value, String name) {
149        return name;
150      }
151    
152      /**
153       * Generate the actual key from the given key/value. The default behavior is that
154       * the actual key is equal to the given key
155       * 
156       * @param key
157       *          the key of the output data
158       * @param value
159       *          the value of the output data
160       * @return the actual key derived from the given key/value
161       */
162      protected K generateActualKey(K key, V value) {
163        return key;
164      }
165      
166      /**
167       * Generate the actual value from the given key and value. The default behavior is that
168       * the actual value is equal to the given value
169       * 
170       * @param key
171       *          the key of the output data
172       * @param value
173       *          the value of the output data
174       * @return the actual value derived from the given key/value
175       */
176      protected V generateActualValue(K key, V value) {
177        return value;
178      }
179      
180    
181      /**
182       * Generate the outfile name based on a given anme and the input file name. If
183       * the {@link JobContext#MAP_INPUT_FILE} does not exists (i.e. this is not for a map only job),
184       * the given name is returned unchanged. If the config value for
185       * "num.of.trailing.legs.to.use" is not set, or set 0 or negative, the given
186       * name is returned unchanged. Otherwise, return a file name consisting of the
187       * N trailing legs of the input file name where N is the config value for
188       * "num.of.trailing.legs.to.use".
189       * 
190       * @param job
191       *          the job config
192       * @param name
193       *          the output file name
194       * @return the outfile name based on a given anme and the input file name.
195       */
196      protected String getInputFileBasedOutputFileName(JobConf job, String name) {
197        String infilepath = job.get(MRJobConfig.MAP_INPUT_FILE);
198        if (infilepath == null) {
199          // if the {@link JobContext#MAP_INPUT_FILE} does not exists,
200          // then return the given name
201          return name;
202        }
203        int numOfTrailingLegsToUse = job.getInt("mapred.outputformat.numOfTrailingLegs", 0);
204        if (numOfTrailingLegsToUse <= 0) {
205          return name;
206        }
207        Path infile = new Path(infilepath);
208        Path parent = infile.getParent();
209        String midName = infile.getName();
210        Path outPath = new Path(midName);
211        for (int i = 1; i < numOfTrailingLegsToUse; i++) {
212          if (parent == null) break;
213          midName = parent.getName();
214          if (midName.length() == 0) break;
215          parent = parent.getParent();
216          outPath = new Path(midName, outPath);
217        }
218        return outPath.toString();
219      }
220    
221      /**
222       * 
223       * @param fs
224       *          the file system to use
225       * @param job
226       *          a job conf object
227       * @param name
228       *          the name of the file over which a record writer object will be
229       *          constructed
230       * @param arg3
231       *          a progressable object
232       * @return A RecordWriter object over the given file
233       * @throws IOException
234       */
235      abstract protected RecordWriter<K, V> getBaseRecordWriter(FileSystem fs,
236          JobConf job, String name, Progressable arg3) throws IOException;
237    }