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    package org.apache.hadoop.mapred;
019    
020    import java.io.IOException;
021    
022    import org.apache.hadoop.classification.InterfaceAudience;
023    import org.apache.hadoop.classification.InterfaceStability;
024    import org.apache.hadoop.fs.FileSystem;
025    import org.apache.hadoop.fs.Path;
026    
027    import org.apache.hadoop.io.WritableComparable;
028    import org.apache.hadoop.io.Writable;
029    import org.apache.hadoop.io.BytesWritable;
030    import org.apache.hadoop.io.SequenceFile;
031    import org.apache.hadoop.io.SequenceFile.CompressionType;
032    import org.apache.hadoop.io.compress.CompressionCodec;
033    import org.apache.hadoop.io.compress.DefaultCodec;
034    import org.apache.hadoop.util.ReflectionUtils;
035    import org.apache.hadoop.util.Progressable;
036    
037    /** 
038     * An {@link OutputFormat} that writes keys, values to 
039     * {@link SequenceFile}s in binary(raw) format
040     * 
041     * @deprecated Use 
042     * {@link org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat}
043     * instead
044     */
045    @Deprecated
046    @InterfaceAudience.Public
047    @InterfaceStability.Stable
048    public class SequenceFileAsBinaryOutputFormat 
049     extends SequenceFileOutputFormat <BytesWritable,BytesWritable> {
050    
051      /** 
052       * Inner class used for appendRaw
053       */
054      static protected class WritableValueBytes extends org.apache.hadoop.mapreduce
055          .lib.output.SequenceFileAsBinaryOutputFormat.WritableValueBytes {
056      }
057    
058      /**
059       * Set the key class for the {@link SequenceFile}
060       * <p>This allows the user to specify the key class to be different 
061       * from the actual class ({@link BytesWritable}) used for writing </p>
062       * 
063       * @param conf the {@link JobConf} to modify
064       * @param theClass the SequenceFile output key class.
065       */
066      static public void setSequenceFileOutputKeyClass(JobConf conf, 
067                                                       Class<?> theClass) {
068        conf.setClass(org.apache.hadoop.mapreduce.lib.output.
069          SequenceFileAsBinaryOutputFormat.KEY_CLASS, theClass, Object.class);
070      }
071    
072      /**
073       * Set the value class for the {@link SequenceFile}
074       * <p>This allows the user to specify the value class to be different 
075       * from the actual class ({@link BytesWritable}) used for writing </p>
076       * 
077       * @param conf the {@link JobConf} to modify
078       * @param theClass the SequenceFile output key class.
079       */
080      static public void setSequenceFileOutputValueClass(JobConf conf, 
081                                                         Class<?> theClass) {
082        conf.setClass(org.apache.hadoop.mapreduce.lib.output.
083          SequenceFileAsBinaryOutputFormat.VALUE_CLASS, theClass, Object.class);
084      }
085    
086      /**
087       * Get the key class for the {@link SequenceFile}
088       * 
089       * @return the key class of the {@link SequenceFile}
090       */
091      static public Class<? extends WritableComparable> getSequenceFileOutputKeyClass(JobConf conf) { 
092        return conf.getClass(org.apache.hadoop.mapreduce.lib.output.
093          SequenceFileAsBinaryOutputFormat.KEY_CLASS, 
094          conf.getOutputKeyClass().asSubclass(WritableComparable.class),
095          WritableComparable.class);
096      }
097    
098      /**
099       * Get the value class for the {@link SequenceFile}
100       * 
101       * @return the value class of the {@link SequenceFile}
102       */
103      static public Class<? extends Writable> getSequenceFileOutputValueClass(JobConf conf) { 
104        return conf.getClass(org.apache.hadoop.mapreduce.lib.output.
105          SequenceFileAsBinaryOutputFormat.VALUE_CLASS, 
106          conf.getOutputValueClass().asSubclass(Writable.class), Writable.class);
107      }
108      
109      @Override 
110      public RecordWriter <BytesWritable, BytesWritable> 
111                 getRecordWriter(FileSystem ignored, JobConf job,
112                                 String name, Progressable progress)
113        throws IOException {
114        // get the path of the temporary output file 
115        Path file = FileOutputFormat.getTaskOutputPath(job, name);
116        
117        FileSystem fs = file.getFileSystem(job);
118        CompressionCodec codec = null;
119        CompressionType compressionType = CompressionType.NONE;
120        if (getCompressOutput(job)) {
121          // find the kind of compression to do
122          compressionType = getOutputCompressionType(job);
123    
124          // find the right codec
125          Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(job,
126              DefaultCodec.class);
127          codec = ReflectionUtils.newInstance(codecClass, job);
128        }
129        final SequenceFile.Writer out = 
130          SequenceFile.createWriter(fs, job, file,
131                        getSequenceFileOutputKeyClass(job),
132                        getSequenceFileOutputValueClass(job),
133                        compressionType,
134                        codec,
135                        progress);
136    
137        return new RecordWriter<BytesWritable, BytesWritable>() {
138            
139            private WritableValueBytes wvaluebytes = new WritableValueBytes();
140    
141            public void write(BytesWritable bkey, BytesWritable bvalue)
142              throws IOException {
143    
144              wvaluebytes.reset(bvalue);
145              out.appendRaw(bkey.getBytes(), 0, bkey.getLength(), wvaluebytes);
146              wvaluebytes.reset(null);
147            }
148    
149            public void close(Reporter reporter) throws IOException { 
150              out.close();
151            }
152    
153          };
154    
155      }
156    
157      @Override 
158      public void checkOutputSpecs(FileSystem ignored, JobConf job) 
159                throws IOException {
160        super.checkOutputSpecs(ignored, job);
161        if (getCompressOutput(job) && 
162            getOutputCompressionType(job) == CompressionType.RECORD ){
163            throw new InvalidJobConfException("SequenceFileAsBinaryOutputFormat "
164                        + "doesn't support Record Compression" );
165        }
166    
167      }
168    
169    }