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.io;
020
021import java.io.*;
022import java.util.*;
023import java.rmi.server.UID;
024import java.security.MessageDigest;
025
026import org.apache.commons.logging.*;
027import org.apache.hadoop.util.Options;
028import org.apache.hadoop.fs.*;
029import org.apache.hadoop.fs.FSDataInputStream.FadviseType;
030import org.apache.hadoop.fs.Options.CreateOpts;
031import org.apache.hadoop.io.compress.CodecPool;
032import org.apache.hadoop.io.compress.CompressionCodec;
033import org.apache.hadoop.io.compress.CompressionInputStream;
034import org.apache.hadoop.io.compress.CompressionOutputStream;
035import org.apache.hadoop.io.compress.Compressor;
036import org.apache.hadoop.io.compress.Decompressor;
037import org.apache.hadoop.io.compress.DefaultCodec;
038import org.apache.hadoop.io.compress.GzipCodec;
039import org.apache.hadoop.io.compress.zlib.ZlibFactory;
040import org.apache.hadoop.io.serializer.Deserializer;
041import org.apache.hadoop.io.serializer.Serializer;
042import org.apache.hadoop.io.serializer.SerializationFactory;
043import org.apache.hadoop.classification.InterfaceAudience;
044import org.apache.hadoop.classification.InterfaceStability;
045import org.apache.hadoop.conf.*;
046import org.apache.hadoop.util.Progressable;
047import org.apache.hadoop.util.Progress;
048import org.apache.hadoop.util.ReflectionUtils;
049import org.apache.hadoop.util.NativeCodeLoader;
050import org.apache.hadoop.util.MergeSort;
051import org.apache.hadoop.util.PriorityQueue;
052import org.apache.hadoop.util.Time;
053
054/** 
055 * <code>SequenceFile</code>s are flat files consisting of binary key/value 
056 * pairs.
057 * 
058 * <p><code>SequenceFile</code> provides {@link Writer}, {@link Reader} and
059 * {@link Sorter} classes for writing, reading and sorting respectively.</p>
060 * 
061 * There are three <code>SequenceFile</code> <code>Writer</code>s based on the 
062 * {@link CompressionType} used to compress key/value pairs:
063 * <ol>
064 *   <li>
065 *   <code>Writer</code> : Uncompressed records.
066 *   </li>
067 *   <li>
068 *   <code>RecordCompressWriter</code> : Record-compressed files, only compress 
069 *                                       values.
070 *   </li>
071 *   <li>
072 *   <code>BlockCompressWriter</code> : Block-compressed files, both keys & 
073 *                                      values are collected in 'blocks' 
074 *                                      separately and compressed. The size of 
075 *                                      the 'block' is configurable.
076 * </ol>
077 * 
078 * <p>The actual compression algorithm used to compress key and/or values can be
079 * specified by using the appropriate {@link CompressionCodec}.</p>
080 * 
081 * <p>The recommended way is to use the static <tt>createWriter</tt> methods
082 * provided by the <code>SequenceFile</code> to chose the preferred format.</p>
083 *
084 * <p>The {@link Reader} acts as the bridge and can read any of the above 
085 * <code>SequenceFile</code> formats.</p>
086 *
087 * <h4 id="Formats">SequenceFile Formats</h4>
088 * 
089 * <p>Essentially there are 3 different formats for <code>SequenceFile</code>s
090 * depending on the <code>CompressionType</code> specified. All of them share a
091 * <a href="#Header">common header</a> described below.
092 * 
093 * <h5 id="Header">SequenceFile Header</h5>
094 * <ul>
095 *   <li>
096 *   version - 3 bytes of magic header <b>SEQ</b>, followed by 1 byte of actual 
097 *             version number (e.g. SEQ4 or SEQ6)
098 *   </li>
099 *   <li>
100 *   keyClassName -key class
101 *   </li>
102 *   <li>
103 *   valueClassName - value class
104 *   </li>
105 *   <li>
106 *   compression - A boolean which specifies if compression is turned on for 
107 *                 keys/values in this file.
108 *   </li>
109 *   <li>
110 *   blockCompression - A boolean which specifies if block-compression is 
111 *                      turned on for keys/values in this file.
112 *   </li>
113 *   <li>
114 *   compression codec - <code>CompressionCodec</code> class which is used for  
115 *                       compression of keys and/or values (if compression is 
116 *                       enabled).
117 *   </li>
118 *   <li>
119 *   metadata - {@link Metadata} for this file.
120 *   </li>
121 *   <li>
122 *   sync - A sync marker to denote end of the header.
123 *   </li>
124 * </ul>
125 * 
126 * <h5 id="#UncompressedFormat">Uncompressed SequenceFile Format</h5>
127 * <ul>
128 * <li>
129 * <a href="#Header">Header</a>
130 * </li>
131 * <li>
132 * Record
133 *   <ul>
134 *     <li>Record length</li>
135 *     <li>Key length</li>
136 *     <li>Key</li>
137 *     <li>Value</li>
138 *   </ul>
139 * </li>
140 * <li>
141 * A sync-marker every few <code>100</code> bytes or so.
142 * </li>
143 * </ul>
144 *
145 * <h5 id="#RecordCompressedFormat">Record-Compressed SequenceFile Format</h5>
146 * <ul>
147 * <li>
148 * <a href="#Header">Header</a>
149 * </li>
150 * <li>
151 * Record
152 *   <ul>
153 *     <li>Record length</li>
154 *     <li>Key length</li>
155 *     <li>Key</li>
156 *     <li><i>Compressed</i> Value</li>
157 *   </ul>
158 * </li>
159 * <li>
160 * A sync-marker every few <code>100</code> bytes or so.
161 * </li>
162 * </ul>
163 * 
164 * <h5 id="#BlockCompressedFormat">Block-Compressed SequenceFile Format</h5>
165 * <ul>
166 * <li>
167 * <a href="#Header">Header</a>
168 * </li>
169 * <li>
170 * Record <i>Block</i>
171 *   <ul>
172 *     <li>Uncompressed number of records in the block</li>
173 *     <li>Compressed key-lengths block-size</li>
174 *     <li>Compressed key-lengths block</li>
175 *     <li>Compressed keys block-size</li>
176 *     <li>Compressed keys block</li>
177 *     <li>Compressed value-lengths block-size</li>
178 *     <li>Compressed value-lengths block</li>
179 *     <li>Compressed values block-size</li>
180 *     <li>Compressed values block</li>
181 *   </ul>
182 * </li>
183 * <li>
184 * A sync-marker every block.
185 * </li>
186 * </ul>
187 * 
188 * <p>The compressed blocks of key lengths and value lengths consist of the 
189 * actual lengths of individual keys/values encoded in ZeroCompressedInteger 
190 * format.</p>
191 * 
192 * @see CompressionCodec
193 */
194@InterfaceAudience.Public
195@InterfaceStability.Stable
196public class SequenceFile {
197  private static final Log LOG = LogFactory.getLog(SequenceFile.class);
198
199  private SequenceFile() {}                         // no public ctor
200
201  private static final byte BLOCK_COMPRESS_VERSION = (byte)4;
202  private static final byte CUSTOM_COMPRESS_VERSION = (byte)5;
203  private static final byte VERSION_WITH_METADATA = (byte)6;
204  private static byte[] VERSION = new byte[] {
205    (byte)'S', (byte)'E', (byte)'Q', VERSION_WITH_METADATA
206  };
207
208  private static final int SYNC_ESCAPE = -1;      // "length" of sync entries
209  private static final int SYNC_HASH_SIZE = 16;   // number of bytes in hash 
210  private static final int SYNC_SIZE = 4+SYNC_HASH_SIZE; // escape + hash
211
212  /** The number of bytes between sync points.*/
213  public static final int SYNC_INTERVAL = 100*SYNC_SIZE; 
214
215  /** 
216   * The compression type used to compress key/value pairs in the 
217   * {@link SequenceFile}.
218   * 
219   * @see SequenceFile.Writer
220   */
221  public static enum CompressionType {
222    /** Do not compress records. */
223    NONE, 
224    /** Compress values only, each separately. */
225    RECORD,
226    /** Compress sequences of records together in blocks. */
227    BLOCK
228  }
229
230  /**
231   * Get the compression type for the reduce outputs
232   * @param job the job config to look in
233   * @return the kind of compression to use
234   */
235  static public CompressionType getDefaultCompressionType(Configuration job) {
236    String name = job.get("io.seqfile.compression.type");
237    return name == null ? CompressionType.RECORD : 
238      CompressionType.valueOf(name);
239  }
240  
241  /**
242   * Set the default compression type for sequence files.
243   * @param job the configuration to modify
244   * @param val the new compression type (none, block, record)
245   */
246  static public void setDefaultCompressionType(Configuration job, 
247                                               CompressionType val) {
248    job.set("io.seqfile.compression.type", val.toString());
249  }
250
251  /**
252   * Create a new Writer with the given options.
253   * @param conf the configuration to use
254   * @param opts the options to create the file with
255   * @return a new Writer
256   * @throws IOException
257   */
258  public static Writer createWriter(Configuration conf, Writer.Option... opts
259                                    ) throws IOException {
260    Writer.CompressionOption compressionOption = 
261      Options.getOption(Writer.CompressionOption.class, opts);
262    CompressionType kind;
263    if (compressionOption != null) {
264      kind = compressionOption.getValue();
265    } else {
266      kind = getDefaultCompressionType(conf);
267      opts = Options.prependOptions(opts, Writer.compression(kind));
268    }
269    switch (kind) {
270      default:
271      case NONE:
272        return new Writer(conf, opts);
273      case RECORD:
274        return new RecordCompressWriter(conf, opts);
275      case BLOCK:
276        return new BlockCompressWriter(conf, opts);
277    }
278  }
279
280  /**
281   * Construct the preferred type of SequenceFile Writer.
282   * @param fs The configured filesystem. 
283   * @param conf The configuration.
284   * @param name The name of the file. 
285   * @param keyClass The 'key' type.
286   * @param valClass The 'value' type.
287   * @return Returns the handle to the constructed SequenceFile Writer.
288   * @throws IOException
289   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
290   *     instead.
291   */
292  @Deprecated
293  public static Writer 
294    createWriter(FileSystem fs, Configuration conf, Path name, 
295                 Class keyClass, Class valClass) throws IOException {
296    return createWriter(conf, Writer.filesystem(fs),
297                        Writer.file(name), Writer.keyClass(keyClass),
298                        Writer.valueClass(valClass));
299  }
300  
301  /**
302   * Construct the preferred type of SequenceFile Writer.
303   * @param fs The configured filesystem. 
304   * @param conf The configuration.
305   * @param name The name of the file. 
306   * @param keyClass The 'key' type.
307   * @param valClass The 'value' type.
308   * @param compressionType The compression type.
309   * @return Returns the handle to the constructed SequenceFile Writer.
310   * @throws IOException
311   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
312   *     instead.
313   */
314  @Deprecated
315  public static Writer 
316    createWriter(FileSystem fs, Configuration conf, Path name, 
317                 Class keyClass, Class valClass, 
318                 CompressionType compressionType) throws IOException {
319    return createWriter(conf, Writer.filesystem(fs),
320                        Writer.file(name), Writer.keyClass(keyClass),
321                        Writer.valueClass(valClass), 
322                        Writer.compression(compressionType));
323  }
324  
325  /**
326   * Construct the preferred type of SequenceFile Writer.
327   * @param fs The configured filesystem. 
328   * @param conf The configuration.
329   * @param name The name of the file. 
330   * @param keyClass The 'key' type.
331   * @param valClass The 'value' type.
332   * @param compressionType The compression type.
333   * @param progress The Progressable object to track progress.
334   * @return Returns the handle to the constructed SequenceFile Writer.
335   * @throws IOException
336   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
337   *     instead.
338   */
339  @Deprecated
340  public static Writer
341    createWriter(FileSystem fs, Configuration conf, Path name, 
342                 Class keyClass, Class valClass, CompressionType compressionType,
343                 Progressable progress) throws IOException {
344    return createWriter(conf, Writer.file(name),
345                        Writer.filesystem(fs),
346                        Writer.keyClass(keyClass),
347                        Writer.valueClass(valClass), 
348                        Writer.compression(compressionType),
349                        Writer.progressable(progress));
350  }
351
352  /**
353   * Construct the preferred type of SequenceFile Writer.
354   * @param fs The configured filesystem. 
355   * @param conf The configuration.
356   * @param name The name of the file. 
357   * @param keyClass The 'key' type.
358   * @param valClass The 'value' type.
359   * @param compressionType The compression type.
360   * @param codec The compression codec.
361   * @return Returns the handle to the constructed SequenceFile Writer.
362   * @throws IOException
363   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
364   *     instead.
365   */
366  @Deprecated
367  public static Writer 
368    createWriter(FileSystem fs, Configuration conf, Path name, 
369                 Class keyClass, Class valClass, CompressionType compressionType, 
370                 CompressionCodec codec) throws IOException {
371    return createWriter(conf, Writer.file(name),
372                        Writer.filesystem(fs),
373                        Writer.keyClass(keyClass),
374                        Writer.valueClass(valClass), 
375                        Writer.compression(compressionType, codec));
376  }
377  
378  /**
379   * Construct the preferred type of SequenceFile Writer.
380   * @param fs The configured filesystem. 
381   * @param conf The configuration.
382   * @param name The name of the file. 
383   * @param keyClass The 'key' type.
384   * @param valClass The 'value' type.
385   * @param compressionType The compression type.
386   * @param codec The compression codec.
387   * @param progress The Progressable object to track progress.
388   * @param metadata The metadata of the file.
389   * @return Returns the handle to the constructed SequenceFile Writer.
390   * @throws IOException
391   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
392   *     instead.
393   */
394  @Deprecated
395  public static Writer
396    createWriter(FileSystem fs, Configuration conf, Path name, 
397                 Class keyClass, Class valClass, 
398                 CompressionType compressionType, CompressionCodec codec,
399                 Progressable progress, Metadata metadata) throws IOException {
400    return createWriter(conf, Writer.file(name),
401                        Writer.filesystem(fs),
402                        Writer.keyClass(keyClass),
403                        Writer.valueClass(valClass),
404                        Writer.compression(compressionType, codec),
405                        Writer.progressable(progress),
406                        Writer.metadata(metadata));
407  }
408
409  /**
410   * Construct the preferred type of SequenceFile Writer.
411   * @param fs The configured filesystem.
412   * @param conf The configuration.
413   * @param name The name of the file.
414   * @param keyClass The 'key' type.
415   * @param valClass The 'value' type.
416   * @param bufferSize buffer size for the underlaying outputstream.
417   * @param replication replication factor for the file.
418   * @param blockSize block size for the file.
419   * @param compressionType The compression type.
420   * @param codec The compression codec.
421   * @param progress The Progressable object to track progress.
422   * @param metadata The metadata of the file.
423   * @return Returns the handle to the constructed SequenceFile Writer.
424   * @throws IOException
425   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
426   *     instead.
427   */
428  @Deprecated
429  public static Writer
430    createWriter(FileSystem fs, Configuration conf, Path name,
431                 Class keyClass, Class valClass, int bufferSize,
432                 short replication, long blockSize,
433                 CompressionType compressionType, CompressionCodec codec,
434                 Progressable progress, Metadata metadata) throws IOException {
435    return createWriter(conf, Writer.file(name),
436                        Writer.filesystem(fs),
437                        Writer.keyClass(keyClass),
438                        Writer.valueClass(valClass), 
439                        Writer.bufferSize(bufferSize), 
440                        Writer.replication(replication),
441                        Writer.blockSize(blockSize),
442                        Writer.compression(compressionType, codec),
443                        Writer.progressable(progress),
444                        Writer.metadata(metadata));
445  }
446
447  /**
448   * Construct the preferred type of SequenceFile Writer.
449   * @param fs The configured filesystem.
450   * @param conf The configuration.
451   * @param name The name of the file.
452   * @param keyClass The 'key' type.
453   * @param valClass The 'value' type.
454   * @param bufferSize buffer size for the underlaying outputstream.
455   * @param replication replication factor for the file.
456   * @param blockSize block size for the file.
457   * @param createParent create parent directory if non-existent
458   * @param compressionType The compression type.
459   * @param codec The compression codec.
460   * @param metadata The metadata of the file.
461   * @return Returns the handle to the constructed SequenceFile Writer.
462   * @throws IOException
463   */
464  @Deprecated
465  public static Writer
466  createWriter(FileSystem fs, Configuration conf, Path name,
467               Class keyClass, Class valClass, int bufferSize,
468               short replication, long blockSize, boolean createParent,
469               CompressionType compressionType, CompressionCodec codec,
470               Metadata metadata) throws IOException {
471    return createWriter(FileContext.getFileContext(fs.getUri(), conf),
472        conf, name, keyClass, valClass, compressionType, codec,
473        metadata, EnumSet.of(CreateFlag.CREATE,CreateFlag.OVERWRITE),
474        CreateOpts.bufferSize(bufferSize),
475        createParent ? CreateOpts.createParent()
476                     : CreateOpts.donotCreateParent(),
477        CreateOpts.repFac(replication),
478        CreateOpts.blockSize(blockSize)
479      );
480  }
481
482  /**
483   * Construct the preferred type of SequenceFile Writer.
484   * @param fc The context for the specified file.
485   * @param conf The configuration.
486   * @param name The name of the file.
487   * @param keyClass The 'key' type.
488   * @param valClass The 'value' type.
489   * @param compressionType The compression type.
490   * @param codec The compression codec.
491   * @param metadata The metadata of the file.
492   * @param createFlag gives the semantics of create: overwrite, append etc.
493   * @param opts file creation options; see {@link CreateOpts}.
494   * @return Returns the handle to the constructed SequenceFile Writer.
495   * @throws IOException
496   */
497  public static Writer
498  createWriter(FileContext fc, Configuration conf, Path name,
499               Class keyClass, Class valClass,
500               CompressionType compressionType, CompressionCodec codec,
501               Metadata metadata,
502               final EnumSet<CreateFlag> createFlag, CreateOpts... opts)
503               throws IOException {
504    return createWriter(conf, fc.create(name, createFlag, opts),
505          keyClass, valClass, compressionType, codec, metadata).ownStream();
506  }
507
508  /**
509   * Construct the preferred type of SequenceFile Writer.
510   * @param fs The configured filesystem. 
511   * @param conf The configuration.
512   * @param name The name of the file. 
513   * @param keyClass The 'key' type.
514   * @param valClass The 'value' type.
515   * @param compressionType The compression type.
516   * @param codec The compression codec.
517   * @param progress The Progressable object to track progress.
518   * @return Returns the handle to the constructed SequenceFile Writer.
519   * @throws IOException
520   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
521   *     instead.
522   */
523  @Deprecated
524  public static Writer
525    createWriter(FileSystem fs, Configuration conf, Path name, 
526                 Class keyClass, Class valClass, 
527                 CompressionType compressionType, CompressionCodec codec,
528                 Progressable progress) throws IOException {
529    return createWriter(conf, Writer.file(name),
530                        Writer.filesystem(fs),
531                        Writer.keyClass(keyClass),
532                        Writer.valueClass(valClass),
533                        Writer.compression(compressionType, codec),
534                        Writer.progressable(progress));
535  }
536
537  /**
538   * Construct the preferred type of 'raw' SequenceFile Writer.
539   * @param conf The configuration.
540   * @param out The stream on top which the writer is to be constructed.
541   * @param keyClass The 'key' type.
542   * @param valClass The 'value' type.
543   * @param compressionType The compression type.
544   * @param codec The compression codec.
545   * @param metadata The metadata of the file.
546   * @return Returns the handle to the constructed SequenceFile Writer.
547   * @throws IOException
548   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
549   *     instead.
550   */
551  @Deprecated
552  public static Writer
553    createWriter(Configuration conf, FSDataOutputStream out, 
554                 Class keyClass, Class valClass,
555                 CompressionType compressionType,
556                 CompressionCodec codec, Metadata metadata) throws IOException {
557    return createWriter(conf, Writer.stream(out), Writer.keyClass(keyClass),
558                        Writer.valueClass(valClass), 
559                        Writer.compression(compressionType, codec),
560                        Writer.metadata(metadata));
561  }
562  
563  /**
564   * Construct the preferred type of 'raw' SequenceFile Writer.
565   * @param conf The configuration.
566   * @param out The stream on top which the writer is to be constructed.
567   * @param keyClass The 'key' type.
568   * @param valClass The 'value' type.
569   * @param compressionType The compression type.
570   * @param codec The compression codec.
571   * @return Returns the handle to the constructed SequenceFile Writer.
572   * @throws IOException
573   * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
574   *     instead.
575   */
576  @Deprecated
577  public static Writer
578    createWriter(Configuration conf, FSDataOutputStream out, 
579                 Class keyClass, Class valClass, CompressionType compressionType,
580                 CompressionCodec codec) throws IOException {
581    return createWriter(conf, Writer.stream(out), Writer.keyClass(keyClass),
582                        Writer.valueClass(valClass),
583                        Writer.compression(compressionType, codec));
584  }
585  
586
587  /** The interface to 'raw' values of SequenceFiles. */
588  public static interface ValueBytes {
589
590    /** Writes the uncompressed bytes to the outStream.
591     * @param outStream : Stream to write uncompressed bytes into.
592     * @throws IOException
593     */
594    public void writeUncompressedBytes(DataOutputStream outStream)
595      throws IOException;
596
597    /** Write compressed bytes to outStream. 
598     * Note: that it will NOT compress the bytes if they are not compressed.
599     * @param outStream : Stream to write compressed bytes into.
600     */
601    public void writeCompressedBytes(DataOutputStream outStream) 
602      throws IllegalArgumentException, IOException;
603
604    /**
605     * Size of stored data.
606     */
607    public int getSize();
608  }
609  
610  private static class UncompressedBytes implements ValueBytes {
611    private int dataSize;
612    private byte[] data;
613    
614    private UncompressedBytes() {
615      data = null;
616      dataSize = 0;
617    }
618    
619    private void reset(DataInputStream in, int length) throws IOException {
620      if (data == null) {
621        data = new byte[length];
622      } else if (length > data.length) {
623        data = new byte[Math.max(length, data.length * 2)];
624      }
625      dataSize = -1;
626      in.readFully(data, 0, length);
627      dataSize = length;
628    }
629    
630    @Override
631    public int getSize() {
632      return dataSize;
633    }
634    
635    @Override
636    public void writeUncompressedBytes(DataOutputStream outStream)
637      throws IOException {
638      outStream.write(data, 0, dataSize);
639    }
640
641    @Override
642    public void writeCompressedBytes(DataOutputStream outStream) 
643      throws IllegalArgumentException, IOException {
644      throw 
645        new IllegalArgumentException("UncompressedBytes cannot be compressed!");
646    }
647
648  } // UncompressedBytes
649  
650  private static class CompressedBytes implements ValueBytes {
651    private int dataSize;
652    private byte[] data;
653    DataInputBuffer rawData = null;
654    CompressionCodec codec = null;
655    CompressionInputStream decompressedStream = null;
656
657    private CompressedBytes(CompressionCodec codec) {
658      data = null;
659      dataSize = 0;
660      this.codec = codec;
661    }
662
663    private void reset(DataInputStream in, int length) throws IOException {
664      if (data == null) {
665        data = new byte[length];
666      } else if (length > data.length) {
667        data = new byte[Math.max(length, data.length * 2)];
668      } 
669      dataSize = -1;
670      in.readFully(data, 0, length);
671      dataSize = length;
672    }
673    
674    @Override
675    public int getSize() {
676      return dataSize;
677    }
678    
679    @Override
680    public void writeUncompressedBytes(DataOutputStream outStream)
681      throws IOException {
682      if (decompressedStream == null) {
683        rawData = new DataInputBuffer();
684        decompressedStream = codec.createInputStream(rawData);
685      } else {
686        decompressedStream.resetState();
687      }
688      rawData.reset(data, 0, dataSize);
689
690      byte[] buffer = new byte[8192];
691      int bytesRead = 0;
692      while ((bytesRead = decompressedStream.read(buffer, 0, 8192)) != -1) {
693        outStream.write(buffer, 0, bytesRead);
694      }
695    }
696
697    @Override
698    public void writeCompressedBytes(DataOutputStream outStream) 
699      throws IllegalArgumentException, IOException {
700      outStream.write(data, 0, dataSize);
701    }
702
703  } // CompressedBytes
704  
705  /**
706   * The class encapsulating with the metadata of a file.
707   * The metadata of a file is a list of attribute name/value
708   * pairs of Text type.
709   *
710   */
711  public static class Metadata implements Writable {
712
713    private TreeMap<Text, Text> theMetadata;
714    
715    public Metadata() {
716      this(new TreeMap<Text, Text>());
717    }
718    
719    public Metadata(TreeMap<Text, Text> arg) {
720      if (arg == null) {
721        this.theMetadata = new TreeMap<Text, Text>();
722      } else {
723        this.theMetadata = arg;
724      }
725    }
726    
727    public Text get(Text name) {
728      return this.theMetadata.get(name);
729    }
730    
731    public void set(Text name, Text value) {
732      this.theMetadata.put(name, value);
733    }
734    
735    public TreeMap<Text, Text> getMetadata() {
736      return new TreeMap<Text, Text>(this.theMetadata);
737    }
738    
739    @Override
740    public void write(DataOutput out) throws IOException {
741      out.writeInt(this.theMetadata.size());
742      Iterator<Map.Entry<Text, Text>> iter =
743        this.theMetadata.entrySet().iterator();
744      while (iter.hasNext()) {
745        Map.Entry<Text, Text> en = iter.next();
746        en.getKey().write(out);
747        en.getValue().write(out);
748      }
749    }
750
751    @Override
752    public void readFields(DataInput in) throws IOException {
753      int sz = in.readInt();
754      if (sz < 0) throw new IOException("Invalid size: " + sz + " for file metadata object");
755      this.theMetadata = new TreeMap<Text, Text>();
756      for (int i = 0; i < sz; i++) {
757        Text key = new Text();
758        Text val = new Text();
759        key.readFields(in);
760        val.readFields(in);
761        this.theMetadata.put(key, val);
762      }    
763    }
764
765    @Override
766    public boolean equals(Object other) {
767      if (other == null) {
768        return false;
769      }
770      if (other.getClass() != this.getClass()) {
771        return false;
772      } else {
773        return equals((Metadata)other);
774      }
775    }
776    
777    public boolean equals(Metadata other) {
778      if (other == null) return false;
779      if (this.theMetadata.size() != other.theMetadata.size()) {
780        return false;
781      }
782      Iterator<Map.Entry<Text, Text>> iter1 =
783        this.theMetadata.entrySet().iterator();
784      Iterator<Map.Entry<Text, Text>> iter2 =
785        other.theMetadata.entrySet().iterator();
786      while (iter1.hasNext() && iter2.hasNext()) {
787        Map.Entry<Text, Text> en1 = iter1.next();
788        Map.Entry<Text, Text> en2 = iter2.next();
789        if (!en1.getKey().equals(en2.getKey())) {
790          return false;
791        }
792        if (!en1.getValue().equals(en2.getValue())) {
793          return false;
794        }
795      }
796      if (iter1.hasNext() || iter2.hasNext()) {
797        return false;
798      }
799      return true;
800    }
801
802    @Override
803    public int hashCode() {
804      assert false : "hashCode not designed";
805      return 42; // any arbitrary constant will do 
806    }
807    
808    @Override
809    public String toString() {
810      StringBuilder sb = new StringBuilder();
811      sb.append("size: ").append(this.theMetadata.size()).append("\n");
812      Iterator<Map.Entry<Text, Text>> iter =
813        this.theMetadata.entrySet().iterator();
814      while (iter.hasNext()) {
815        Map.Entry<Text, Text> en = iter.next();
816        sb.append("\t").append(en.getKey().toString()).append("\t").append(en.getValue().toString());
817        sb.append("\n");
818      }
819      return sb.toString();
820    }
821  }
822  
823  /** Write key/value pairs to a sequence-format file. */
824  public static class Writer implements java.io.Closeable, Syncable {
825    private Configuration conf;
826    FSDataOutputStream out;
827    boolean ownOutputStream = true;
828    DataOutputBuffer buffer = new DataOutputBuffer();
829
830    Class keyClass;
831    Class valClass;
832
833    private final CompressionType compress;
834    CompressionCodec codec = null;
835    CompressionOutputStream deflateFilter = null;
836    DataOutputStream deflateOut = null;
837    Metadata metadata = null;
838    Compressor compressor = null;
839    
840    protected Serializer keySerializer;
841    protected Serializer uncompressedValSerializer;
842    protected Serializer compressedValSerializer;
843    
844    // Insert a globally unique 16-byte value every few entries, so that one
845    // can seek into the middle of a file and then synchronize with record
846    // starts and ends by scanning for this value.
847    long lastSyncPos;                     // position of last sync
848    byte[] sync;                          // 16 random bytes
849    {
850      try {                                       
851        MessageDigest digester = MessageDigest.getInstance("MD5");
852        long time = Time.now();
853        digester.update((new UID()+"@"+time).getBytes());
854        sync = digester.digest();
855      } catch (Exception e) {
856        throw new RuntimeException(e);
857      }
858    }
859
860    public static interface Option {}
861    
862    static class FileOption extends Options.PathOption 
863                                    implements Option {
864      FileOption(Path path) {
865        super(path);
866      }
867    }
868
869    /**
870     * @deprecated only used for backwards-compatibility in the createWriter methods
871     * that take FileSystem.
872     */
873    @Deprecated
874    private static class FileSystemOption implements Option {
875      private final FileSystem value;
876      protected FileSystemOption(FileSystem value) {
877        this.value = value;
878      }
879      public FileSystem getValue() {
880        return value;
881      }
882    }
883
884    static class StreamOption extends Options.FSDataOutputStreamOption 
885                              implements Option {
886      StreamOption(FSDataOutputStream stream) {
887        super(stream);
888      }
889    }
890
891    static class BufferSizeOption extends Options.IntegerOption
892                                  implements Option {
893      BufferSizeOption(int value) {
894        super(value);
895      }
896    }
897    
898    static class BlockSizeOption extends Options.LongOption implements Option {
899      BlockSizeOption(long value) {
900        super(value);
901      }
902    }
903
904    static class ReplicationOption extends Options.IntegerOption
905                                   implements Option {
906      ReplicationOption(int value) {
907        super(value);
908      }
909    }
910
911    static class KeyClassOption extends Options.ClassOption implements Option {
912      KeyClassOption(Class<?> value) {
913        super(value);
914      }
915    }
916
917    static class ValueClassOption extends Options.ClassOption
918                                          implements Option {
919      ValueClassOption(Class<?> value) {
920        super(value);
921      }
922    }
923
924    static class MetadataOption implements Option {
925      private final Metadata value;
926      MetadataOption(Metadata value) {
927        this.value = value;
928      }
929      Metadata getValue() {
930        return value;
931      }
932    }
933
934    static class ProgressableOption extends Options.ProgressableOption
935                                    implements Option {
936      ProgressableOption(Progressable value) {
937        super(value);
938      }
939    }
940
941    private static class CompressionOption implements Option {
942      private final CompressionType value;
943      private final CompressionCodec codec;
944      CompressionOption(CompressionType value) {
945        this(value, null);
946      }
947      CompressionOption(CompressionType value, CompressionCodec codec) {
948        this.value = value;
949        this.codec = (CompressionType.NONE != value && null == codec)
950          ? new DefaultCodec()
951          : codec;
952      }
953      CompressionType getValue() {
954        return value;
955      }
956      CompressionCodec getCodec() {
957        return codec;
958      }
959    }
960    
961    public static Option file(Path value) {
962      return new FileOption(value);
963    }
964
965    /**
966     * @deprecated only used for backwards-compatibility in the createWriter methods
967     * that take FileSystem.
968     */
969    @Deprecated
970    private static Option filesystem(FileSystem fs) {
971      return new SequenceFile.Writer.FileSystemOption(fs);
972    }
973    
974    public static Option bufferSize(int value) {
975      return new BufferSizeOption(value);
976    }
977    
978    public static Option stream(FSDataOutputStream value) {
979      return new StreamOption(value);
980    }
981    
982    public static Option replication(short value) {
983      return new ReplicationOption(value);
984    }
985    
986    public static Option blockSize(long value) {
987      return new BlockSizeOption(value);
988    }
989    
990    public static Option progressable(Progressable value) {
991      return new ProgressableOption(value);
992    }
993
994    public static Option keyClass(Class<?> value) {
995      return new KeyClassOption(value);
996    }
997    
998    public static Option valueClass(Class<?> value) {
999      return new ValueClassOption(value);
1000    }
1001    
1002    public static Option metadata(Metadata value) {
1003      return new MetadataOption(value);
1004    }
1005
1006    public static Option compression(CompressionType value) {
1007      return new CompressionOption(value);
1008    }
1009
1010    public static Option compression(CompressionType value,
1011        CompressionCodec codec) {
1012      return new CompressionOption(value, codec);
1013    }
1014    
1015    /**
1016     * Construct a uncompressed writer from a set of options.
1017     * @param conf the configuration to use
1018     * @param options the options used when creating the writer
1019     * @throws IOException if it fails
1020     */
1021    Writer(Configuration conf, 
1022           Option... opts) throws IOException {
1023      BlockSizeOption blockSizeOption = 
1024        Options.getOption(BlockSizeOption.class, opts);
1025      BufferSizeOption bufferSizeOption = 
1026        Options.getOption(BufferSizeOption.class, opts);
1027      ReplicationOption replicationOption = 
1028        Options.getOption(ReplicationOption.class, opts);
1029      ProgressableOption progressOption = 
1030        Options.getOption(ProgressableOption.class, opts);
1031      FileOption fileOption = Options.getOption(FileOption.class, opts);
1032      FileSystemOption fsOption = Options.getOption(FileSystemOption.class, opts);
1033      StreamOption streamOption = Options.getOption(StreamOption.class, opts);
1034      KeyClassOption keyClassOption = 
1035        Options.getOption(KeyClassOption.class, opts);
1036      ValueClassOption valueClassOption = 
1037        Options.getOption(ValueClassOption.class, opts);
1038      MetadataOption metadataOption = 
1039        Options.getOption(MetadataOption.class, opts);
1040      CompressionOption compressionTypeOption =
1041        Options.getOption(CompressionOption.class, opts);
1042      // check consistency of options
1043      if ((fileOption == null) == (streamOption == null)) {
1044        throw new IllegalArgumentException("file or stream must be specified");
1045      }
1046      if (fileOption == null && (blockSizeOption != null ||
1047                                 bufferSizeOption != null ||
1048                                 replicationOption != null ||
1049                                 progressOption != null)) {
1050        throw new IllegalArgumentException("file modifier options not " +
1051                                           "compatible with stream");
1052      }
1053
1054      FSDataOutputStream out;
1055      boolean ownStream = fileOption != null;
1056      if (ownStream) {
1057        Path p = fileOption.getValue();
1058        FileSystem fs;
1059        if (fsOption != null) {
1060          fs = fsOption.getValue();
1061        } else {
1062          fs = p.getFileSystem(conf);
1063        }
1064        int bufferSize = bufferSizeOption == null ? getBufferSize(conf) :
1065          bufferSizeOption.getValue();
1066        short replication = replicationOption == null ? 
1067          fs.getDefaultReplication(p) :
1068          (short) replicationOption.getValue();
1069        long blockSize = blockSizeOption == null ? fs.getDefaultBlockSize(p) :
1070          blockSizeOption.getValue();
1071        Progressable progress = progressOption == null ? null :
1072          progressOption.getValue();
1073        out = fs.create(p, true, bufferSize, replication, blockSize, progress);
1074      } else {
1075        out = streamOption.getValue();
1076      }
1077      Class<?> keyClass = keyClassOption == null ?
1078          Object.class : keyClassOption.getValue();
1079      Class<?> valueClass = valueClassOption == null ?
1080          Object.class : valueClassOption.getValue();
1081      Metadata metadata = metadataOption == null ?
1082          new Metadata() : metadataOption.getValue();
1083      this.compress = compressionTypeOption.getValue();
1084      final CompressionCodec codec = compressionTypeOption.getCodec();
1085      if (codec != null &&
1086          (codec instanceof GzipCodec) &&
1087          !NativeCodeLoader.isNativeCodeLoaded() &&
1088          !ZlibFactory.isNativeZlibLoaded(conf)) {
1089        throw new IllegalArgumentException("SequenceFile doesn't work with " +
1090                                           "GzipCodec without native-hadoop " +
1091                                           "code!");
1092      }
1093      init(conf, out, ownStream, keyClass, valueClass, codec, metadata);
1094    }
1095
1096    /** Create the named file.
1097     * @deprecated Use 
1098     *   {@link SequenceFile#createWriter(Configuration, Writer.Option...)} 
1099     *   instead.
1100     */
1101    @Deprecated
1102    public Writer(FileSystem fs, Configuration conf, Path name, 
1103                  Class keyClass, Class valClass) throws IOException {
1104      this.compress = CompressionType.NONE;
1105      init(conf, fs.create(name), true, keyClass, valClass, null, 
1106           new Metadata());
1107    }
1108    
1109    /** Create the named file with write-progress reporter.
1110     * @deprecated Use 
1111     *   {@link SequenceFile#createWriter(Configuration, Writer.Option...)} 
1112     *   instead.
1113     */
1114    @Deprecated
1115    public Writer(FileSystem fs, Configuration conf, Path name, 
1116                  Class keyClass, Class valClass,
1117                  Progressable progress, Metadata metadata) throws IOException {
1118      this.compress = CompressionType.NONE;
1119      init(conf, fs.create(name, progress), true, keyClass, valClass,
1120           null, metadata);
1121    }
1122    
1123    /** Create the named file with write-progress reporter. 
1124     * @deprecated Use 
1125     *   {@link SequenceFile#createWriter(Configuration, Writer.Option...)} 
1126     *   instead.
1127     */
1128    @Deprecated
1129    public Writer(FileSystem fs, Configuration conf, Path name,
1130                  Class keyClass, Class valClass,
1131                  int bufferSize, short replication, long blockSize,
1132                  Progressable progress, Metadata metadata) throws IOException {
1133      this.compress = CompressionType.NONE;
1134      init(conf,
1135           fs.create(name, true, bufferSize, replication, blockSize, progress),
1136           true, keyClass, valClass, null, metadata);
1137    }
1138
1139    boolean isCompressed() { return compress != CompressionType.NONE; }
1140    boolean isBlockCompressed() { return compress == CompressionType.BLOCK; }
1141    
1142    Writer ownStream() { this.ownOutputStream = true; return this;  }
1143
1144    /** Write and flush the file header. */
1145    private void writeFileHeader() 
1146      throws IOException {
1147      out.write(VERSION);
1148      Text.writeString(out, keyClass.getName());
1149      Text.writeString(out, valClass.getName());
1150      
1151      out.writeBoolean(this.isCompressed());
1152      out.writeBoolean(this.isBlockCompressed());
1153      
1154      if (this.isCompressed()) {
1155        Text.writeString(out, (codec.getClass()).getName());
1156      }
1157      this.metadata.write(out);
1158      out.write(sync);                       // write the sync bytes
1159      out.flush();                           // flush header
1160    }
1161    
1162    /** Initialize. */
1163    @SuppressWarnings("unchecked")
1164    void init(Configuration conf, FSDataOutputStream out, boolean ownStream,
1165              Class keyClass, Class valClass,
1166              CompressionCodec codec, Metadata metadata) 
1167      throws IOException {
1168      this.conf = conf;
1169      this.out = out;
1170      this.ownOutputStream = ownStream;
1171      this.keyClass = keyClass;
1172      this.valClass = valClass;
1173      this.codec = codec;
1174      this.metadata = metadata;
1175      SerializationFactory serializationFactory = new SerializationFactory(conf);
1176      this.keySerializer = serializationFactory.getSerializer(keyClass);
1177      if (this.keySerializer == null) {
1178        throw new IOException(
1179            "Could not find a serializer for the Key class: '"
1180                + keyClass.getCanonicalName() + "'. "
1181                + "Please ensure that the configuration '" +
1182                CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is "
1183                + "properly configured, if you're using"
1184                + "custom serialization.");
1185      }
1186      this.keySerializer.open(buffer);
1187      this.uncompressedValSerializer = serializationFactory.getSerializer(valClass);
1188      if (this.uncompressedValSerializer == null) {
1189        throw new IOException(
1190            "Could not find a serializer for the Value class: '"
1191                + valClass.getCanonicalName() + "'. "
1192                + "Please ensure that the configuration '" +
1193                CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is "
1194                + "properly configured, if you're using"
1195                + "custom serialization.");
1196      }
1197      this.uncompressedValSerializer.open(buffer);
1198      if (this.codec != null) {
1199        ReflectionUtils.setConf(this.codec, this.conf);
1200        this.compressor = CodecPool.getCompressor(this.codec);
1201        this.deflateFilter = this.codec.createOutputStream(buffer, compressor);
1202        this.deflateOut = 
1203          new DataOutputStream(new BufferedOutputStream(deflateFilter));
1204        this.compressedValSerializer = serializationFactory.getSerializer(valClass);
1205        if (this.compressedValSerializer == null) {
1206          throw new IOException(
1207              "Could not find a serializer for the Value class: '"
1208                  + valClass.getCanonicalName() + "'. "
1209                  + "Please ensure that the configuration '" +
1210                  CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is "
1211                  + "properly configured, if you're using"
1212                  + "custom serialization.");
1213        }
1214        this.compressedValSerializer.open(deflateOut);
1215      }
1216      writeFileHeader();
1217    }
1218    
1219    /** Returns the class of keys in this file. */
1220    public Class getKeyClass() { return keyClass; }
1221
1222    /** Returns the class of values in this file. */
1223    public Class getValueClass() { return valClass; }
1224
1225    /** Returns the compression codec of data in this file. */
1226    public CompressionCodec getCompressionCodec() { return codec; }
1227    
1228    /** create a sync point */
1229    public void sync() throws IOException {
1230      if (sync != null && lastSyncPos != out.getPos()) {
1231        out.writeInt(SYNC_ESCAPE);                // mark the start of the sync
1232        out.write(sync);                          // write sync
1233        lastSyncPos = out.getPos();               // update lastSyncPos
1234      }
1235    }
1236
1237    /**
1238     * flush all currently written data to the file system
1239     * @deprecated Use {@link #hsync()} or {@link #hflush()} instead
1240     */
1241    @Deprecated
1242    public void syncFs() throws IOException {
1243      if (out != null) {
1244        out.sync();                               // flush contents to file system
1245      }
1246    }
1247
1248    @Override
1249    public void hsync() throws IOException {
1250      if (out != null) {
1251        out.hsync();
1252      }
1253    }
1254
1255    @Override
1256    public void hflush() throws IOException {
1257      if (out != null) {
1258        out.hflush();
1259      }
1260    }
1261    
1262    /** Returns the configuration of this file. */
1263    Configuration getConf() { return conf; }
1264    
1265    /** Close the file. */
1266    @Override
1267    public synchronized void close() throws IOException {
1268      keySerializer.close();
1269      uncompressedValSerializer.close();
1270      if (compressedValSerializer != null) {
1271        compressedValSerializer.close();
1272      }
1273
1274      CodecPool.returnCompressor(compressor);
1275      compressor = null;
1276      
1277      if (out != null) {
1278        
1279        // Close the underlying stream iff we own it...
1280        if (ownOutputStream) {
1281          out.close();
1282        } else {
1283          out.flush();
1284        }
1285        out = null;
1286      }
1287    }
1288
1289    synchronized void checkAndWriteSync() throws IOException {
1290      if (sync != null &&
1291          out.getPos() >= lastSyncPos+SYNC_INTERVAL) { // time to emit sync
1292        sync();
1293      }
1294    }
1295
1296    /** Append a key/value pair. */
1297    public void append(Writable key, Writable val)
1298      throws IOException {
1299      append((Object) key, (Object) val);
1300    }
1301
1302    /** Append a key/value pair. */
1303    @SuppressWarnings("unchecked")
1304    public synchronized void append(Object key, Object val)
1305      throws IOException {
1306      if (key.getClass() != keyClass)
1307        throw new IOException("wrong key class: "+key.getClass().getName()
1308                              +" is not "+keyClass);
1309      if (val.getClass() != valClass)
1310        throw new IOException("wrong value class: "+val.getClass().getName()
1311                              +" is not "+valClass);
1312
1313      buffer.reset();
1314
1315      // Append the 'key'
1316      keySerializer.serialize(key);
1317      int keyLength = buffer.getLength();
1318      if (keyLength < 0)
1319        throw new IOException("negative length keys not allowed: " + key);
1320
1321      // Append the 'value'
1322      if (compress == CompressionType.RECORD) {
1323        deflateFilter.resetState();
1324        compressedValSerializer.serialize(val);
1325        deflateOut.flush();
1326        deflateFilter.finish();
1327      } else {
1328        uncompressedValSerializer.serialize(val);
1329      }
1330
1331      // Write the record out
1332      checkAndWriteSync();                                // sync
1333      out.writeInt(buffer.getLength());                   // total record length
1334      out.writeInt(keyLength);                            // key portion length
1335      out.write(buffer.getData(), 0, buffer.getLength()); // data
1336    }
1337
1338    public synchronized void appendRaw(byte[] keyData, int keyOffset,
1339        int keyLength, ValueBytes val) throws IOException {
1340      if (keyLength < 0)
1341        throw new IOException("negative length keys not allowed: " + keyLength);
1342
1343      int valLength = val.getSize();
1344
1345      checkAndWriteSync();
1346      
1347      out.writeInt(keyLength+valLength);          // total record length
1348      out.writeInt(keyLength);                    // key portion length
1349      out.write(keyData, keyOffset, keyLength);   // key
1350      val.writeUncompressedBytes(out);            // value
1351    }
1352
1353    /** Returns the current length of the output file.
1354     *
1355     * <p>This always returns a synchronized position.  In other words,
1356     * immediately after calling {@link SequenceFile.Reader#seek(long)} with a position
1357     * returned by this method, {@link SequenceFile.Reader#next(Writable)} may be called.  However
1358     * the key may be earlier in the file than key last written when this
1359     * method was called (e.g., with block-compression, it may be the first key
1360     * in the block that was being written when this method was called).
1361     */
1362    public synchronized long getLength() throws IOException {
1363      return out.getPos();
1364    }
1365
1366  } // class Writer
1367
1368  /** Write key/compressed-value pairs to a sequence-format file. */
1369  static class RecordCompressWriter extends Writer {
1370    
1371    RecordCompressWriter(Configuration conf, 
1372                         Option... options) throws IOException {
1373      super(conf, options);
1374    }
1375
1376    /** Append a key/value pair. */
1377    @Override
1378    @SuppressWarnings("unchecked")
1379    public synchronized void append(Object key, Object val)
1380      throws IOException {
1381      if (key.getClass() != keyClass)
1382        throw new IOException("wrong key class: "+key.getClass().getName()
1383                              +" is not "+keyClass);
1384      if (val.getClass() != valClass)
1385        throw new IOException("wrong value class: "+val.getClass().getName()
1386                              +" is not "+valClass);
1387
1388      buffer.reset();
1389
1390      // Append the 'key'
1391      keySerializer.serialize(key);
1392      int keyLength = buffer.getLength();
1393      if (keyLength < 0)
1394        throw new IOException("negative length keys not allowed: " + key);
1395
1396      // Compress 'value' and append it
1397      deflateFilter.resetState();
1398      compressedValSerializer.serialize(val);
1399      deflateOut.flush();
1400      deflateFilter.finish();
1401
1402      // Write the record out
1403      checkAndWriteSync();                                // sync
1404      out.writeInt(buffer.getLength());                   // total record length
1405      out.writeInt(keyLength);                            // key portion length
1406      out.write(buffer.getData(), 0, buffer.getLength()); // data
1407    }
1408
1409    /** Append a key/value pair. */
1410    @Override
1411    public synchronized void appendRaw(byte[] keyData, int keyOffset,
1412        int keyLength, ValueBytes val) throws IOException {
1413
1414      if (keyLength < 0)
1415        throw new IOException("negative length keys not allowed: " + keyLength);
1416
1417      int valLength = val.getSize();
1418      
1419      checkAndWriteSync();                        // sync
1420      out.writeInt(keyLength+valLength);          // total record length
1421      out.writeInt(keyLength);                    // key portion length
1422      out.write(keyData, keyOffset, keyLength);   // 'key' data
1423      val.writeCompressedBytes(out);              // 'value' data
1424    }
1425    
1426  } // RecordCompressionWriter
1427
1428  /** Write compressed key/value blocks to a sequence-format file. */
1429  static class BlockCompressWriter extends Writer {
1430    
1431    private int noBufferedRecords = 0;
1432    
1433    private DataOutputBuffer keyLenBuffer = new DataOutputBuffer();
1434    private DataOutputBuffer keyBuffer = new DataOutputBuffer();
1435
1436    private DataOutputBuffer valLenBuffer = new DataOutputBuffer();
1437    private DataOutputBuffer valBuffer = new DataOutputBuffer();
1438
1439    private final int compressionBlockSize;
1440    
1441    BlockCompressWriter(Configuration conf,
1442                        Option... options) throws IOException {
1443      super(conf, options);
1444      compressionBlockSize = 
1445        conf.getInt("io.seqfile.compress.blocksize", 1000000);
1446      keySerializer.close();
1447      keySerializer.open(keyBuffer);
1448      uncompressedValSerializer.close();
1449      uncompressedValSerializer.open(valBuffer);
1450    }
1451
1452    /** Workhorse to check and write out compressed data/lengths */
1453    private synchronized 
1454      void writeBuffer(DataOutputBuffer uncompressedDataBuffer) 
1455      throws IOException {
1456      deflateFilter.resetState();
1457      buffer.reset();
1458      deflateOut.write(uncompressedDataBuffer.getData(), 0, 
1459                       uncompressedDataBuffer.getLength());
1460      deflateOut.flush();
1461      deflateFilter.finish();
1462      
1463      WritableUtils.writeVInt(out, buffer.getLength());
1464      out.write(buffer.getData(), 0, buffer.getLength());
1465    }
1466    
1467    /** Compress and flush contents to dfs */
1468    @Override
1469    public synchronized void sync() throws IOException {
1470      if (noBufferedRecords > 0) {
1471        super.sync();
1472        
1473        // No. of records
1474        WritableUtils.writeVInt(out, noBufferedRecords);
1475        
1476        // Write 'keys' and lengths
1477        writeBuffer(keyLenBuffer);
1478        writeBuffer(keyBuffer);
1479        
1480        // Write 'values' and lengths
1481        writeBuffer(valLenBuffer);
1482        writeBuffer(valBuffer);
1483        
1484        // Flush the file-stream
1485        out.flush();
1486        
1487        // Reset internal states
1488        keyLenBuffer.reset();
1489        keyBuffer.reset();
1490        valLenBuffer.reset();
1491        valBuffer.reset();
1492        noBufferedRecords = 0;
1493      }
1494      
1495    }
1496    
1497    /** Close the file. */
1498    @Override
1499    public synchronized void close() throws IOException {
1500      if (out != null) {
1501        sync();
1502      }
1503      super.close();
1504    }
1505
1506    /** Append a key/value pair. */
1507    @Override
1508    @SuppressWarnings("unchecked")
1509    public synchronized void append(Object key, Object val)
1510      throws IOException {
1511      if (key.getClass() != keyClass)
1512        throw new IOException("wrong key class: "+key+" is not "+keyClass);
1513      if (val.getClass() != valClass)
1514        throw new IOException("wrong value class: "+val+" is not "+valClass);
1515
1516      // Save key/value into respective buffers 
1517      int oldKeyLength = keyBuffer.getLength();
1518      keySerializer.serialize(key);
1519      int keyLength = keyBuffer.getLength() - oldKeyLength;
1520      if (keyLength < 0)
1521        throw new IOException("negative length keys not allowed: " + key);
1522      WritableUtils.writeVInt(keyLenBuffer, keyLength);
1523
1524      int oldValLength = valBuffer.getLength();
1525      uncompressedValSerializer.serialize(val);
1526      int valLength = valBuffer.getLength() - oldValLength;
1527      WritableUtils.writeVInt(valLenBuffer, valLength);
1528      
1529      // Added another key/value pair
1530      ++noBufferedRecords;
1531      
1532      // Compress and flush?
1533      int currentBlockSize = keyBuffer.getLength() + valBuffer.getLength();
1534      if (currentBlockSize >= compressionBlockSize) {
1535        sync();
1536      }
1537    }
1538    
1539    /** Append a key/value pair. */
1540    @Override
1541    public synchronized void appendRaw(byte[] keyData, int keyOffset,
1542        int keyLength, ValueBytes val) throws IOException {
1543      
1544      if (keyLength < 0)
1545        throw new IOException("negative length keys not allowed");
1546
1547      int valLength = val.getSize();
1548      
1549      // Save key/value data in relevant buffers
1550      WritableUtils.writeVInt(keyLenBuffer, keyLength);
1551      keyBuffer.write(keyData, keyOffset, keyLength);
1552      WritableUtils.writeVInt(valLenBuffer, valLength);
1553      val.writeUncompressedBytes(valBuffer);
1554
1555      // Added another key/value pair
1556      ++noBufferedRecords;
1557
1558      // Compress and flush?
1559      int currentBlockSize = keyBuffer.getLength() + valBuffer.getLength(); 
1560      if (currentBlockSize >= compressionBlockSize) {
1561        sync();
1562      }
1563    }
1564  
1565  } // BlockCompressionWriter
1566
1567  /** Get the configured buffer size */
1568  private static int getBufferSize(Configuration conf) {
1569    return conf.getInt("io.file.buffer.size", 4096);
1570  }
1571
1572  /** Reads key/value pairs from a sequence-format file. */
1573  public static class Reader implements java.io.Closeable {
1574    private String filename;
1575    private FSDataInputStream in;
1576    private DataOutputBuffer outBuf = new DataOutputBuffer();
1577
1578    private byte version;
1579
1580    private String keyClassName;
1581    private String valClassName;
1582    private Class keyClass;
1583    private Class valClass;
1584
1585    private CompressionCodec codec = null;
1586    private Metadata metadata = null;
1587    
1588    private byte[] sync = new byte[SYNC_HASH_SIZE];
1589    private byte[] syncCheck = new byte[SYNC_HASH_SIZE];
1590    private boolean syncSeen;
1591
1592    private long headerEnd;
1593    private long start;
1594    private long end;
1595    private int keyLength;
1596    private int recordLength;
1597
1598    private boolean decompress;
1599    private boolean blockCompressed;
1600    
1601    private Configuration conf;
1602
1603    private int noBufferedRecords = 0;
1604    private boolean lazyDecompress = true;
1605    private boolean valuesDecompressed = true;
1606    
1607    private int noBufferedKeys = 0;
1608    private int noBufferedValues = 0;
1609    
1610    private DataInputBuffer keyLenBuffer = null;
1611    private CompressionInputStream keyLenInFilter = null;
1612    private DataInputStream keyLenIn = null;
1613    private Decompressor keyLenDecompressor = null;
1614    private DataInputBuffer keyBuffer = null;
1615    private CompressionInputStream keyInFilter = null;
1616    private DataInputStream keyIn = null;
1617    private Decompressor keyDecompressor = null;
1618
1619    private DataInputBuffer valLenBuffer = null;
1620    private CompressionInputStream valLenInFilter = null;
1621    private DataInputStream valLenIn = null;
1622    private Decompressor valLenDecompressor = null;
1623    private DataInputBuffer valBuffer = null;
1624    private CompressionInputStream valInFilter = null;
1625    private DataInputStream valIn = null;
1626    private Decompressor valDecompressor = null;
1627    
1628    private Deserializer keyDeserializer;
1629    private Deserializer valDeserializer;
1630
1631    /**
1632     * A tag interface for all of the Reader options
1633     */
1634    public static interface Option {}
1635    
1636    /**
1637     * Create an option to specify the path name of the sequence file.
1638     * @param value the path to read
1639     * @return a new option
1640     */
1641    public static Option file(Path value) {
1642      return new FileOption(value);
1643    }
1644    
1645    /**
1646     * Create an option to specify the stream with the sequence file.
1647     * @param value the stream to read.
1648     * @return a new option
1649     */
1650    public static Option stream(FSDataInputStream value) {
1651      return new InputStreamOption(value);
1652    }
1653    
1654    /**
1655     * Create an option to specify the starting byte to read.
1656     * @param value the number of bytes to skip over
1657     * @return a new option
1658     */
1659    public static Option start(long value) {
1660      return new StartOption(value);
1661    }
1662    
1663    /**
1664     * Create an option to specify the number of bytes to read.
1665     * @param value the number of bytes to read
1666     * @return a new option
1667     */
1668    public static Option length(long value) {
1669      return new LengthOption(value);
1670    }
1671    
1672    /**
1673     * Create an option with the buffer size for reading the given pathname.
1674     * @param value the number of bytes to buffer
1675     * @return a new option
1676     */
1677    public static Option bufferSize(int value) {
1678      return new BufferSizeOption(value);
1679    }
1680
1681    private static class FileOption extends Options.PathOption 
1682                                    implements Option {
1683      private FileOption(Path value) {
1684        super(value);
1685      }
1686    }
1687    
1688    private static class InputStreamOption
1689        extends Options.FSDataInputStreamOption 
1690        implements Option {
1691      private InputStreamOption(FSDataInputStream value) {
1692        super(value);
1693      }
1694    }
1695
1696    private static class StartOption extends Options.LongOption
1697                                     implements Option {
1698      private StartOption(long value) {
1699        super(value);
1700      }
1701    }
1702
1703    private static class LengthOption extends Options.LongOption
1704                                      implements Option {
1705      private LengthOption(long value) {
1706        super(value);
1707      }
1708    }
1709
1710    private static class BufferSizeOption extends Options.IntegerOption
1711                                      implements Option {
1712      private BufferSizeOption(int value) {
1713        super(value);
1714      }
1715    }
1716
1717    // only used directly
1718    private static class OnlyHeaderOption extends Options.BooleanOption 
1719                                          implements Option {
1720      private OnlyHeaderOption() {
1721        super(true);
1722      }
1723    }
1724
1725    public Reader(Configuration conf, Option... opts) throws IOException {
1726      // Look up the options, these are null if not set
1727      FileOption fileOpt = Options.getOption(FileOption.class, opts);
1728      InputStreamOption streamOpt = 
1729        Options.getOption(InputStreamOption.class, opts);
1730      StartOption startOpt = Options.getOption(StartOption.class, opts);
1731      LengthOption lenOpt = Options.getOption(LengthOption.class, opts);
1732      BufferSizeOption bufOpt = Options.getOption(BufferSizeOption.class,opts);
1733      OnlyHeaderOption headerOnly = 
1734        Options.getOption(OnlyHeaderOption.class, opts);
1735      // check for consistency
1736      if ((fileOpt == null) == (streamOpt == null)) {
1737        throw new 
1738          IllegalArgumentException("File or stream option must be specified");
1739      }
1740      if (fileOpt == null && bufOpt != null) {
1741        throw new IllegalArgumentException("buffer size can only be set when" +
1742                                           " a file is specified.");
1743      }
1744      // figure out the real values
1745      Path filename = null;
1746      FSDataInputStream file;
1747      final long len;
1748      if (fileOpt != null) {
1749        filename = fileOpt.getValue();
1750        FileSystem fs = filename.getFileSystem(conf);
1751        int bufSize = bufOpt == null ? getBufferSize(conf): bufOpt.getValue();
1752        len = null == lenOpt
1753          ? fs.getFileStatus(filename).getLen()
1754          : lenOpt.getValue();
1755        file = openFile(fs, filename, bufSize, len);
1756      } else {
1757        len = null == lenOpt ? Long.MAX_VALUE : lenOpt.getValue();
1758        file = streamOpt.getValue();
1759      }
1760      long start = startOpt == null ? 0 : startOpt.getValue();
1761      // really set up
1762      initialize(filename, file, start, len, conf, headerOnly != null);
1763    }
1764
1765    /**
1766     * Construct a reader by opening a file from the given file system.
1767     * @param fs The file system used to open the file.
1768     * @param file The file being read.
1769     * @param conf Configuration
1770     * @throws IOException
1771     * @deprecated Use Reader(Configuration, Option...) instead.
1772     */
1773    @Deprecated
1774    public Reader(FileSystem fs, Path file, 
1775                  Configuration conf) throws IOException {
1776      this(conf, file(file.makeQualified(fs)));
1777    }
1778
1779    /**
1780     * Construct a reader by the given input stream.
1781     * @param in An input stream.
1782     * @param buffersize unused
1783     * @param start The starting position.
1784     * @param length The length being read.
1785     * @param conf Configuration
1786     * @throws IOException
1787     * @deprecated Use Reader(Configuration, Reader.Option...) instead.
1788     */
1789    @Deprecated
1790    public Reader(FSDataInputStream in, int buffersize,
1791        long start, long length, Configuration conf) throws IOException {
1792      this(conf, stream(in), start(start), length(length));
1793    }
1794
1795    /** Common work of the constructors. */
1796    private void initialize(Path filename, FSDataInputStream in,
1797                            long start, long length, Configuration conf,
1798                            boolean tempReader) throws IOException {
1799      if (in == null) {
1800        throw new IllegalArgumentException("in == null");
1801      }
1802      this.filename = filename == null ? "<unknown>" : filename.toString();
1803      this.in = in;
1804      this.conf = conf;
1805      boolean succeeded = false;
1806      this.start = start;
1807      try {
1808        seek(start);
1809        this.end = this.in.getPos() + length;
1810        // if it wrapped around, use the max
1811        if (end < length) {
1812          end = Long.MAX_VALUE;
1813        }
1814        init(tempReader);
1815        succeeded = true;
1816      } finally {
1817        if (!succeeded) {
1818          IOUtils.cleanup(LOG, this.in);
1819        }
1820      }
1821    }
1822
1823    /**
1824     * Override this method to specialize the type of
1825     * {@link FSDataInputStream} returned.
1826     * @param fs The file system used to open the file.
1827     * @param file The file being read.
1828     * @param bufferSize The buffer size used to read the file.
1829     * @param length The length being read if it is >= 0.  Otherwise,
1830     *               the length is not available.
1831     * @return The opened stream.
1832     * @throws IOException
1833     */
1834    protected FSDataInputStream openFile(FileSystem fs, Path file,
1835        int bufferSize, long length) throws IOException {
1836      return fs.open(file, bufferSize);
1837    }
1838    
1839    /**
1840     * Initialize the {@link Reader}
1841     * @param tmpReader <code>true</code> if we are constructing a temporary
1842     *                  reader {@link SequenceFile.Sorter.cloneFileAttributes}, 
1843     *                  and hence do not initialize every component; 
1844     *                  <code>false</code> otherwise.
1845     * @throws IOException
1846     */
1847    private void init(boolean tempReader) throws IOException {
1848      byte[] versionBlock = new byte[VERSION.length];
1849      in.readFully(versionBlock);
1850
1851      if ((versionBlock[0] != VERSION[0]) ||
1852          (versionBlock[1] != VERSION[1]) ||
1853          (versionBlock[2] != VERSION[2]))
1854        throw new IOException(this + " not a SequenceFile");
1855
1856      // Set 'version'
1857      version = versionBlock[3];
1858      if (version > VERSION[3])
1859        throw new VersionMismatchException(VERSION[3], version);
1860
1861      if (version < BLOCK_COMPRESS_VERSION) {
1862        UTF8 className = new UTF8();
1863
1864        className.readFields(in);
1865        keyClassName = className.toStringChecked(); // key class name
1866
1867        className.readFields(in);
1868        valClassName = className.toStringChecked(); // val class name
1869      } else {
1870        keyClassName = Text.readString(in);
1871        valClassName = Text.readString(in);
1872      }
1873
1874      if (version > 2) {                          // if version > 2
1875        this.decompress = in.readBoolean();       // is compressed?
1876      } else {
1877        decompress = false;
1878      }
1879
1880      if (version >= BLOCK_COMPRESS_VERSION) {    // if version >= 4
1881        this.blockCompressed = in.readBoolean();  // is block-compressed?
1882      } else {
1883        blockCompressed = false;
1884      }
1885      
1886      // if version >= 5
1887      // setup the compression codec
1888      if (decompress) {
1889        if (version >= CUSTOM_COMPRESS_VERSION) {
1890          String codecClassname = Text.readString(in);
1891          try {
1892            Class<? extends CompressionCodec> codecClass
1893              = conf.getClassByName(codecClassname).asSubclass(CompressionCodec.class);
1894            this.codec = ReflectionUtils.newInstance(codecClass, conf);
1895          } catch (ClassNotFoundException cnfe) {
1896            throw new IllegalArgumentException("Unknown codec: " + 
1897                                               codecClassname, cnfe);
1898          }
1899        } else {
1900          codec = new DefaultCodec();
1901          ((Configurable)codec).setConf(conf);
1902        }
1903      }
1904      
1905      this.metadata = new Metadata();
1906      if (version >= VERSION_WITH_METADATA) {    // if version >= 6
1907        this.metadata.readFields(in);
1908      }
1909      
1910      if (version > 1) {                          // if version > 1
1911        in.readFully(sync);                       // read sync bytes
1912        headerEnd = in.getPos();                  // record end of header
1913      }
1914      
1915      // Initialize... *not* if this we are constructing a temporary Reader
1916      if (!tempReader) {
1917        valBuffer = new DataInputBuffer();
1918        if (decompress) {
1919          valDecompressor = CodecPool.getDecompressor(codec);
1920          valInFilter = codec.createInputStream(valBuffer, valDecompressor);
1921          valIn = new DataInputStream(valInFilter);
1922        } else {
1923          valIn = valBuffer;
1924        }
1925
1926        if (blockCompressed) {
1927          keyLenBuffer = new DataInputBuffer();
1928          keyBuffer = new DataInputBuffer();
1929          valLenBuffer = new DataInputBuffer();
1930
1931          keyLenDecompressor = CodecPool.getDecompressor(codec);
1932          keyLenInFilter = codec.createInputStream(keyLenBuffer, 
1933                                                   keyLenDecompressor);
1934          keyLenIn = new DataInputStream(keyLenInFilter);
1935
1936          keyDecompressor = CodecPool.getDecompressor(codec);
1937          keyInFilter = codec.createInputStream(keyBuffer, keyDecompressor);
1938          keyIn = new DataInputStream(keyInFilter);
1939
1940          valLenDecompressor = CodecPool.getDecompressor(codec);
1941          valLenInFilter = codec.createInputStream(valLenBuffer, 
1942                                                   valLenDecompressor);
1943          valLenIn = new DataInputStream(valLenInFilter);
1944        }
1945        
1946        SerializationFactory serializationFactory =
1947          new SerializationFactory(conf);
1948        this.keyDeserializer =
1949          getDeserializer(serializationFactory, getKeyClass());
1950        if (this.keyDeserializer == null) {
1951          throw new IOException(
1952              "Could not find a deserializer for the Key class: '"
1953                  + getKeyClass().getCanonicalName() + "'. "
1954                  + "Please ensure that the configuration '" +
1955                  CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is "
1956                  + "properly configured, if you're using "
1957                  + "custom serialization.");
1958        }
1959        if (!blockCompressed) {
1960          this.keyDeserializer.open(valBuffer);
1961        } else {
1962          this.keyDeserializer.open(keyIn);
1963        }
1964        this.valDeserializer =
1965          getDeserializer(serializationFactory, getValueClass());
1966        if (this.valDeserializer == null) {
1967          throw new IOException(
1968              "Could not find a deserializer for the Value class: '"
1969                  + getValueClass().getCanonicalName() + "'. "
1970                  + "Please ensure that the configuration '" +
1971                  CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is "
1972                  + "properly configured, if you're using "
1973                  + "custom serialization.");
1974        }
1975        this.valDeserializer.open(valIn);
1976      }
1977    }
1978    
1979    @SuppressWarnings("unchecked")
1980    private Deserializer getDeserializer(SerializationFactory sf, Class c) {
1981      return sf.getDeserializer(c);
1982    }
1983    
1984    /** Close the file. */
1985    @Override
1986    public synchronized void close() throws IOException {
1987      // Return the decompressors to the pool
1988      CodecPool.returnDecompressor(keyLenDecompressor);
1989      CodecPool.returnDecompressor(keyDecompressor);
1990      CodecPool.returnDecompressor(valLenDecompressor);
1991      CodecPool.returnDecompressor(valDecompressor);
1992      keyLenDecompressor = keyDecompressor = null;
1993      valLenDecompressor = valDecompressor = null;
1994      
1995      if (keyDeserializer != null) {
1996        keyDeserializer.close();
1997      }
1998      if (valDeserializer != null) {
1999        valDeserializer.close();
2000      }
2001
2002      try {
2003        in.adviseFile(FadviseType.FILE_DONTNEED, start, end);
2004      } catch (IOException ioe) {
2005        if (LOG.isInfoEnabled()) {
2006          LOG.info("Error in fadvise. Ignoring it.", ioe);
2007        }
2008      }
2009
2010      // Close the input-stream
2011      in.close();
2012    }
2013
2014    /** Returns the name of the key class. */
2015    public String getKeyClassName() {
2016      return keyClassName;
2017    }
2018
2019    /** Returns the class of keys in this file. */
2020    public synchronized Class<?> getKeyClass() {
2021      if (null == keyClass) {
2022        try {
2023          keyClass = WritableName.getClass(getKeyClassName(), conf);
2024        } catch (IOException e) {
2025          throw new RuntimeException(e);
2026        }
2027      }
2028      return keyClass;
2029    }
2030
2031    /** Returns the name of the value class. */
2032    public String getValueClassName() {
2033      return valClassName;
2034    }
2035
2036    /** Returns the class of values in this file. */
2037    public synchronized Class<?> getValueClass() {
2038      if (null == valClass) {
2039        try {
2040          valClass = WritableName.getClass(getValueClassName(), conf);
2041        } catch (IOException e) {
2042          throw new RuntimeException(e);
2043        }
2044      }
2045      return valClass;
2046    }
2047
2048    /** Returns true if values are compressed. */
2049    public boolean isCompressed() { return decompress; }
2050    
2051    /** Returns true if records are block-compressed. */
2052    public boolean isBlockCompressed() { return blockCompressed; }
2053    
2054    /** Returns the compression codec of data in this file. */
2055    public CompressionCodec getCompressionCodec() { return codec; }
2056    
2057    /**
2058     * Get the compression type for this file.
2059     * @return the compression type
2060     */
2061    public CompressionType getCompressionType() {
2062      if (decompress) {
2063        return blockCompressed ? CompressionType.BLOCK : CompressionType.RECORD;
2064      } else {
2065        return CompressionType.NONE;
2066      }
2067    }
2068
2069    /** Returns the metadata object of the file */
2070    public Metadata getMetadata() {
2071      return this.metadata;
2072    }
2073    
2074    /** Returns the configuration used for this file. */
2075    Configuration getConf() { return conf; }
2076    
2077    /** Read a compressed buffer */
2078    private synchronized void readBuffer(DataInputBuffer buffer, 
2079                                         CompressionInputStream filter) throws IOException {
2080      // Read data into a temporary buffer
2081      DataOutputBuffer dataBuffer = new DataOutputBuffer();
2082
2083      try {
2084        int dataBufferLength = WritableUtils.readVInt(in);
2085        dataBuffer.write(in, dataBufferLength);
2086      
2087        // Set up 'buffer' connected to the input-stream
2088        buffer.reset(dataBuffer.getData(), 0, dataBuffer.getLength());
2089      } finally {
2090        dataBuffer.close();
2091      }
2092
2093      // Reset the codec
2094      filter.resetState();
2095    }
2096    
2097    /** Read the next 'compressed' block */
2098    private synchronized void readBlock() throws IOException {
2099      // Check if we need to throw away a whole block of 
2100      // 'values' due to 'lazy decompression' 
2101      if (lazyDecompress && !valuesDecompressed) {
2102        in.seek(WritableUtils.readVInt(in)+in.getPos());
2103        in.seek(WritableUtils.readVInt(in)+in.getPos());
2104      }
2105      
2106      // Reset internal states
2107      noBufferedKeys = 0; noBufferedValues = 0; noBufferedRecords = 0;
2108      valuesDecompressed = false;
2109
2110      //Process sync
2111      if (sync != null) {
2112        in.readInt();
2113        in.readFully(syncCheck);                // read syncCheck
2114        if (!Arrays.equals(sync, syncCheck))    // check it
2115          throw new IOException("File is corrupt!");
2116      }
2117      syncSeen = true;
2118
2119      // Read number of records in this block
2120      noBufferedRecords = WritableUtils.readVInt(in);
2121      
2122      // Read key lengths and keys
2123      readBuffer(keyLenBuffer, keyLenInFilter);
2124      readBuffer(keyBuffer, keyInFilter);
2125      noBufferedKeys = noBufferedRecords;
2126      
2127      // Read value lengths and values
2128      if (!lazyDecompress) {
2129        readBuffer(valLenBuffer, valLenInFilter);
2130        readBuffer(valBuffer, valInFilter);
2131        noBufferedValues = noBufferedRecords;
2132        valuesDecompressed = true;
2133      }
2134    }
2135
2136    /** 
2137     * Position valLenIn/valIn to the 'value' 
2138     * corresponding to the 'current' key 
2139     */
2140    private synchronized void seekToCurrentValue() throws IOException {
2141      if (!blockCompressed) {
2142        if (decompress) {
2143          valInFilter.resetState();
2144        }
2145        valBuffer.reset();
2146      } else {
2147        // Check if this is the first value in the 'block' to be read
2148        if (lazyDecompress && !valuesDecompressed) {
2149          // Read the value lengths and values
2150          readBuffer(valLenBuffer, valLenInFilter);
2151          readBuffer(valBuffer, valInFilter);
2152          noBufferedValues = noBufferedRecords;
2153          valuesDecompressed = true;
2154        }
2155        
2156        // Calculate the no. of bytes to skip
2157        // Note: 'current' key has already been read!
2158        int skipValBytes = 0;
2159        int currentKey = noBufferedKeys + 1;          
2160        for (int i=noBufferedValues; i > currentKey; --i) {
2161          skipValBytes += WritableUtils.readVInt(valLenIn);
2162          --noBufferedValues;
2163        }
2164        
2165        // Skip to the 'val' corresponding to 'current' key
2166        if (skipValBytes > 0) {
2167          if (valIn.skipBytes(skipValBytes) != skipValBytes) {
2168            throw new IOException("Failed to seek to " + currentKey + 
2169                                  "(th) value!");
2170          }
2171        }
2172      }
2173    }
2174
2175    /**
2176     * Get the 'value' corresponding to the last read 'key'.
2177     * @param val : The 'value' to be read.
2178     * @throws IOException
2179     */
2180    public synchronized void getCurrentValue(Writable val) 
2181      throws IOException {
2182      if (val instanceof Configurable) {
2183        ((Configurable) val).setConf(this.conf);
2184      }
2185
2186      // Position stream to 'current' value
2187      seekToCurrentValue();
2188
2189      if (!blockCompressed) {
2190        val.readFields(valIn);
2191        
2192        if (valIn.read() > 0) {
2193          LOG.info("available bytes: " + valIn.available());
2194          throw new IOException(val+" read "+(valBuffer.getPosition()-keyLength)
2195                                + " bytes, should read " +
2196                                (valBuffer.getLength()-keyLength));
2197        }
2198      } else {
2199        // Get the value
2200        int valLength = WritableUtils.readVInt(valLenIn);
2201        val.readFields(valIn);
2202        
2203        // Read another compressed 'value'
2204        --noBufferedValues;
2205        
2206        // Sanity check
2207        if ((valLength < 0) && LOG.isDebugEnabled()) {
2208          LOG.debug(val + " is a zero-length value");
2209        }
2210      }
2211
2212    }
2213    
2214    /**
2215     * Get the 'value' corresponding to the last read 'key'.
2216     * @param val : The 'value' to be read.
2217     * @throws IOException
2218     */
2219    public synchronized Object getCurrentValue(Object val) 
2220      throws IOException {
2221      if (val instanceof Configurable) {
2222        ((Configurable) val).setConf(this.conf);
2223      }
2224
2225      // Position stream to 'current' value
2226      seekToCurrentValue();
2227
2228      if (!blockCompressed) {
2229        val = deserializeValue(val);
2230        
2231        if (valIn.read() > 0) {
2232          LOG.info("available bytes: " + valIn.available());
2233          throw new IOException(val+" read "+(valBuffer.getPosition()-keyLength)
2234                                + " bytes, should read " +
2235                                (valBuffer.getLength()-keyLength));
2236        }
2237      } else {
2238        // Get the value
2239        int valLength = WritableUtils.readVInt(valLenIn);
2240        val = deserializeValue(val);
2241        
2242        // Read another compressed 'value'
2243        --noBufferedValues;
2244        
2245        // Sanity check
2246        if ((valLength < 0) && LOG.isDebugEnabled()) {
2247          LOG.debug(val + " is a zero-length value");
2248        }
2249      }
2250      return val;
2251
2252    }
2253
2254    @SuppressWarnings("unchecked")
2255    private Object deserializeValue(Object val) throws IOException {
2256      return valDeserializer.deserialize(val);
2257    }
2258    
2259    /** Read the next key in the file into <code>key</code>, skipping its
2260     * value.  True if another entry exists, and false at end of file. */
2261    public synchronized boolean next(Writable key) throws IOException {
2262      if (key.getClass() != getKeyClass())
2263        throw new IOException("wrong key class: "+key.getClass().getName()
2264                              +" is not "+keyClass);
2265
2266      if (!blockCompressed) {
2267        outBuf.reset();
2268        
2269        keyLength = next(outBuf);
2270        if (keyLength < 0)
2271          return false;
2272        
2273        valBuffer.reset(outBuf.getData(), outBuf.getLength());
2274        
2275        key.readFields(valBuffer);
2276        valBuffer.mark(0);
2277        if (valBuffer.getPosition() != keyLength)
2278          throw new IOException(key + " read " + valBuffer.getPosition()
2279                                + " bytes, should read " + keyLength);
2280      } else {
2281        //Reset syncSeen
2282        syncSeen = false;
2283        
2284        if (noBufferedKeys == 0) {
2285          try {
2286            readBlock();
2287          } catch (EOFException eof) {
2288            return false;
2289          }
2290        }
2291        
2292        int keyLength = WritableUtils.readVInt(keyLenIn);
2293        
2294        // Sanity check
2295        if (keyLength < 0) {
2296          return false;
2297        }
2298        
2299        //Read another compressed 'key'
2300        key.readFields(keyIn);
2301        --noBufferedKeys;
2302      }
2303
2304      return true;
2305    }
2306
2307    /** Read the next key/value pair in the file into <code>key</code> and
2308     * <code>val</code>.  Returns true if such a pair exists and false when at
2309     * end of file */
2310    public synchronized boolean next(Writable key, Writable val)
2311      throws IOException {
2312      if (val.getClass() != getValueClass())
2313        throw new IOException("wrong value class: "+val+" is not "+valClass);
2314
2315      boolean more = next(key);
2316      
2317      if (more) {
2318        getCurrentValue(val);
2319      }
2320
2321      return more;
2322    }
2323    
2324    /**
2325     * Read and return the next record length, potentially skipping over 
2326     * a sync block.
2327     * @return the length of the next record or -1 if there is no next record
2328     * @throws IOException
2329     */
2330    private synchronized int readRecordLength() throws IOException {
2331      if (in.getPos() >= end) {
2332        return -1;
2333      }      
2334      int length = in.readInt();
2335      if (version > 1 && sync != null &&
2336          length == SYNC_ESCAPE) {              // process a sync entry
2337        in.readFully(syncCheck);                // read syncCheck
2338        if (!Arrays.equals(sync, syncCheck))    // check it
2339          throw new IOException("File is corrupt!");
2340        syncSeen = true;
2341        if (in.getPos() >= end) {
2342          return -1;
2343        }
2344        length = in.readInt();                  // re-read length
2345      } else {
2346        syncSeen = false;
2347      }
2348      
2349      return length;
2350    }
2351    
2352    /** Read the next key/value pair in the file into <code>buffer</code>.
2353     * Returns the length of the key read, or -1 if at end of file.  The length
2354     * of the value may be computed by calling buffer.getLength() before and
2355     * after calls to this method. */
2356    /** @deprecated Call {@link #nextRaw(DataOutputBuffer,SequenceFile.ValueBytes)}. */
2357    @Deprecated
2358    synchronized int next(DataOutputBuffer buffer) throws IOException {
2359      // Unsupported for block-compressed sequence files
2360      if (blockCompressed) {
2361        throw new IOException("Unsupported call for block-compressed" +
2362                              " SequenceFiles - use SequenceFile.Reader.next(DataOutputStream, ValueBytes)");
2363      }
2364      try {
2365        int length = readRecordLength();
2366        if (length == -1) {
2367          return -1;
2368        }
2369        int keyLength = in.readInt();
2370        buffer.write(in, length);
2371        return keyLength;
2372      } catch (ChecksumException e) {             // checksum failure
2373        handleChecksumException(e);
2374        return next(buffer);
2375      }
2376    }
2377
2378    public ValueBytes createValueBytes() {
2379      ValueBytes val = null;
2380      if (!decompress || blockCompressed) {
2381        val = new UncompressedBytes();
2382      } else {
2383        val = new CompressedBytes(codec);
2384      }
2385      return val;
2386    }
2387
2388    /**
2389     * Read 'raw' records.
2390     * @param key - The buffer into which the key is read
2391     * @param val - The 'raw' value
2392     * @return Returns the total record length or -1 for end of file
2393     * @throws IOException
2394     */
2395    public synchronized int nextRaw(DataOutputBuffer key, ValueBytes val) 
2396      throws IOException {
2397      if (!blockCompressed) {
2398        int length = readRecordLength();
2399        if (length == -1) {
2400          return -1;
2401        }
2402        int keyLength = in.readInt();
2403        int valLength = length - keyLength;
2404        key.write(in, keyLength);
2405        if (decompress) {
2406          CompressedBytes value = (CompressedBytes)val;
2407          value.reset(in, valLength);
2408        } else {
2409          UncompressedBytes value = (UncompressedBytes)val;
2410          value.reset(in, valLength);
2411        }
2412        
2413        return length;
2414      } else {
2415        //Reset syncSeen
2416        syncSeen = false;
2417        
2418        // Read 'key'
2419        if (noBufferedKeys == 0) {
2420          if (in.getPos() >= end) 
2421            return -1;
2422
2423          try { 
2424            readBlock();
2425          } catch (EOFException eof) {
2426            return -1;
2427          }
2428        }
2429        int keyLength = WritableUtils.readVInt(keyLenIn);
2430        if (keyLength < 0) {
2431          throw new IOException("zero length key found!");
2432        }
2433        key.write(keyIn, keyLength);
2434        --noBufferedKeys;
2435        
2436        // Read raw 'value'
2437        seekToCurrentValue();
2438        int valLength = WritableUtils.readVInt(valLenIn);
2439        UncompressedBytes rawValue = (UncompressedBytes)val;
2440        rawValue.reset(valIn, valLength);
2441        --noBufferedValues;
2442        
2443        return (keyLength+valLength);
2444      }
2445      
2446    }
2447
2448    /**
2449     * Read 'raw' keys.
2450     * @param key - The buffer into which the key is read
2451     * @return Returns the key length or -1 for end of file
2452     * @throws IOException
2453     */
2454    public synchronized int nextRawKey(DataOutputBuffer key) 
2455      throws IOException {
2456      if (!blockCompressed) {
2457        recordLength = readRecordLength();
2458        if (recordLength == -1) {
2459          return -1;
2460        }
2461        keyLength = in.readInt();
2462        key.write(in, keyLength);
2463        return keyLength;
2464      } else {
2465        //Reset syncSeen
2466        syncSeen = false;
2467        
2468        // Read 'key'
2469        if (noBufferedKeys == 0) {
2470          if (in.getPos() >= end) 
2471            return -1;
2472
2473          try { 
2474            readBlock();
2475          } catch (EOFException eof) {
2476            return -1;
2477          }
2478        }
2479        int keyLength = WritableUtils.readVInt(keyLenIn);
2480        if (keyLength < 0) {
2481          throw new IOException("zero length key found!");
2482        }
2483        key.write(keyIn, keyLength);
2484        --noBufferedKeys;
2485        
2486        return keyLength;
2487      }
2488      
2489    }
2490
2491    /** Read the next key in the file, skipping its
2492     * value.  Return null at end of file. */
2493    public synchronized Object next(Object key) throws IOException {
2494      if (key != null && key.getClass() != getKeyClass()) {
2495        throw new IOException("wrong key class: "+key.getClass().getName()
2496                              +" is not "+keyClass);
2497      }
2498
2499      if (!blockCompressed) {
2500        outBuf.reset();
2501        
2502        keyLength = next(outBuf);
2503        if (keyLength < 0)
2504          return null;
2505        
2506        valBuffer.reset(outBuf.getData(), outBuf.getLength());
2507        
2508        key = deserializeKey(key);
2509        valBuffer.mark(0);
2510        if (valBuffer.getPosition() != keyLength)
2511          throw new IOException(key + " read " + valBuffer.getPosition()
2512                                + " bytes, should read " + keyLength);
2513      } else {
2514        //Reset syncSeen
2515        syncSeen = false;
2516        
2517        if (noBufferedKeys == 0) {
2518          try {
2519            readBlock();
2520          } catch (EOFException eof) {
2521            return null;
2522          }
2523        }
2524        
2525        int keyLength = WritableUtils.readVInt(keyLenIn);
2526        
2527        // Sanity check
2528        if (keyLength < 0) {
2529          return null;
2530        }
2531        
2532        //Read another compressed 'key'
2533        key = deserializeKey(key);
2534        --noBufferedKeys;
2535      }
2536
2537      return key;
2538    }
2539
2540    @SuppressWarnings("unchecked")
2541    private Object deserializeKey(Object key) throws IOException {
2542      return keyDeserializer.deserialize(key);
2543    }
2544
2545    /**
2546     * Read 'raw' values.
2547     * @param val - The 'raw' value
2548     * @return Returns the value length
2549     * @throws IOException
2550     */
2551    public synchronized int nextRawValue(ValueBytes val) 
2552      throws IOException {
2553      
2554      // Position stream to current value
2555      seekToCurrentValue();
2556 
2557      if (!blockCompressed) {
2558        int valLength = recordLength - keyLength;
2559        if (decompress) {
2560          CompressedBytes value = (CompressedBytes)val;
2561          value.reset(in, valLength);
2562        } else {
2563          UncompressedBytes value = (UncompressedBytes)val;
2564          value.reset(in, valLength);
2565        }
2566         
2567        return valLength;
2568      } else {
2569        int valLength = WritableUtils.readVInt(valLenIn);
2570        UncompressedBytes rawValue = (UncompressedBytes)val;
2571        rawValue.reset(valIn, valLength);
2572        --noBufferedValues;
2573        return valLength;
2574      }
2575      
2576    }
2577
2578    private void handleChecksumException(ChecksumException e)
2579      throws IOException {
2580      if (this.conf.getBoolean("io.skip.checksum.errors", false)) {
2581        LOG.warn("Bad checksum at "+getPosition()+". Skipping entries.");
2582        sync(getPosition()+this.conf.getInt("io.bytes.per.checksum", 512));
2583      } else {
2584        throw e;
2585      }
2586    }
2587
2588    /** disables sync. often invoked for tmp files */
2589    synchronized void ignoreSync() {
2590      sync = null;
2591    }
2592    
2593    /** Set the current byte position in the input file.
2594     *
2595     * <p>The position passed must be a position returned by {@link
2596     * SequenceFile.Writer#getLength()} when writing this file.  To seek to an arbitrary
2597     * position, use {@link SequenceFile.Reader#sync(long)}.
2598     */
2599    public synchronized void seek(long position) throws IOException {
2600      in.seek(position);
2601      if (blockCompressed) {                      // trigger block read
2602        noBufferedKeys = 0;
2603        valuesDecompressed = true;
2604      }
2605    }
2606
2607    /** Seek to the next sync mark past a given position.*/
2608    public synchronized void sync(long position) throws IOException {
2609      if (position+SYNC_SIZE >= end) {
2610        seek(end);
2611        return;
2612      }
2613
2614      if (position < headerEnd) {
2615        // seek directly to first record
2616        in.seek(headerEnd);
2617        // note the sync marker "seen" in the header
2618        syncSeen = true;
2619        return;
2620      }
2621
2622      try {
2623        seek(position+4);                         // skip escape
2624        in.readFully(syncCheck);
2625        int syncLen = sync.length;
2626        for (int i = 0; in.getPos() < end; i++) {
2627          int j = 0;
2628          for (; j < syncLen; j++) {
2629            if (sync[j] != syncCheck[(i+j)%syncLen])
2630              break;
2631          }
2632          if (j == syncLen) {
2633            in.seek(in.getPos() - SYNC_SIZE);     // position before sync
2634            return;
2635          }
2636          syncCheck[i%syncLen] = in.readByte();
2637        }
2638      } catch (ChecksumException e) {             // checksum failure
2639        handleChecksumException(e);
2640      }
2641    }
2642
2643    /** Returns true iff the previous call to next passed a sync mark.*/
2644    public synchronized boolean syncSeen() { return syncSeen; }
2645
2646    /** Return the current byte position in the input file. */
2647    public synchronized long getPosition() throws IOException {
2648      return in.getPos();
2649    }
2650
2651    /** Returns the name of the file. */
2652    @Override
2653    public String toString() {
2654      return filename;
2655    }
2656
2657  }
2658
2659  /** Sorts key/value pairs in a sequence-format file.
2660   *
2661   * <p>For best performance, applications should make sure that the {@link
2662   * Writable#readFields(DataInput)} implementation of their keys is
2663   * very efficient.  In particular, it should avoid allocating memory.
2664   */
2665  public static class Sorter {
2666
2667    private RawComparator comparator;
2668
2669    private MergeSort mergeSort; //the implementation of merge sort
2670    
2671    private Path[] inFiles;                     // when merging or sorting
2672
2673    private Path outFile;
2674
2675    private int memory; // bytes
2676    private int factor; // merged per pass
2677
2678    private FileSystem fs = null;
2679
2680    private Class keyClass;
2681    private Class valClass;
2682
2683    private Configuration conf;
2684    private Metadata metadata;
2685    
2686    private Progressable progressable = null;
2687
2688    /** Sort and merge files containing the named classes. */
2689    public Sorter(FileSystem fs, Class<? extends WritableComparable> keyClass,
2690                  Class valClass, Configuration conf)  {
2691      this(fs, WritableComparator.get(keyClass), keyClass, valClass, conf);
2692    }
2693
2694    /** Sort and merge using an arbitrary {@link RawComparator}. */
2695    public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, 
2696                  Class valClass, Configuration conf) {
2697      this(fs, comparator, keyClass, valClass, conf, new Metadata());
2698    }
2699
2700    /** Sort and merge using an arbitrary {@link RawComparator}. */
2701    public Sorter(FileSystem fs, RawComparator comparator, Class keyClass,
2702                  Class valClass, Configuration conf, Metadata metadata) {
2703      this.fs = fs;
2704      this.comparator = comparator;
2705      this.keyClass = keyClass;
2706      this.valClass = valClass;
2707      this.memory = conf.getInt("io.sort.mb", 100) * 1024 * 1024;
2708      this.factor = conf.getInt("io.sort.factor", 100);
2709      this.conf = conf;
2710      this.metadata = metadata;
2711    }
2712
2713    /** Set the number of streams to merge at once.*/
2714    public void setFactor(int factor) { this.factor = factor; }
2715
2716    /** Get the number of streams to merge at once.*/
2717    public int getFactor() { return factor; }
2718
2719    /** Set the total amount of buffer memory, in bytes.*/
2720    public void setMemory(int memory) { this.memory = memory; }
2721
2722    /** Get the total amount of buffer memory, in bytes.*/
2723    public int getMemory() { return memory; }
2724
2725    /** Set the progressable object in order to report progress. */
2726    public void setProgressable(Progressable progressable) {
2727      this.progressable = progressable;
2728    }
2729    
2730    /** 
2731     * Perform a file sort from a set of input files into an output file.
2732     * @param inFiles the files to be sorted
2733     * @param outFile the sorted output file
2734     * @param deleteInput should the input files be deleted as they are read?
2735     */
2736    public void sort(Path[] inFiles, Path outFile,
2737                     boolean deleteInput) throws IOException {
2738      if (fs.exists(outFile)) {
2739        throw new IOException("already exists: " + outFile);
2740      }
2741
2742      this.inFiles = inFiles;
2743      this.outFile = outFile;
2744
2745      int segments = sortPass(deleteInput);
2746      if (segments > 1) {
2747        mergePass(outFile.getParent());
2748      }
2749    }
2750
2751    /** 
2752     * Perform a file sort from a set of input files and return an iterator.
2753     * @param inFiles the files to be sorted
2754     * @param tempDir the directory where temp files are created during sort
2755     * @param deleteInput should the input files be deleted as they are read?
2756     * @return iterator the RawKeyValueIterator
2757     */
2758    public RawKeyValueIterator sortAndIterate(Path[] inFiles, Path tempDir, 
2759                                              boolean deleteInput) throws IOException {
2760      Path outFile = new Path(tempDir + Path.SEPARATOR + "all.2");
2761      if (fs.exists(outFile)) {
2762        throw new IOException("already exists: " + outFile);
2763      }
2764      this.inFiles = inFiles;
2765      //outFile will basically be used as prefix for temp files in the cases
2766      //where sort outputs multiple sorted segments. For the single segment
2767      //case, the outputFile itself will contain the sorted data for that
2768      //segment
2769      this.outFile = outFile;
2770
2771      int segments = sortPass(deleteInput);
2772      if (segments > 1)
2773        return merge(outFile.suffix(".0"), outFile.suffix(".0.index"), 
2774                     tempDir);
2775      else if (segments == 1)
2776        return merge(new Path[]{outFile}, true, tempDir);
2777      else return null;
2778    }
2779
2780    /**
2781     * The backwards compatible interface to sort.
2782     * @param inFile the input file to sort
2783     * @param outFile the sorted output file
2784     */
2785    public void sort(Path inFile, Path outFile) throws IOException {
2786      sort(new Path[]{inFile}, outFile, false);
2787    }
2788    
2789    private int sortPass(boolean deleteInput) throws IOException {
2790      if(LOG.isDebugEnabled()) {
2791        LOG.debug("running sort pass");
2792      }
2793      SortPass sortPass = new SortPass();         // make the SortPass
2794      sortPass.setProgressable(progressable);
2795      mergeSort = new MergeSort(sortPass.new SeqFileComparator());
2796      try {
2797        return sortPass.run(deleteInput);         // run it
2798      } finally {
2799        sortPass.close();                         // close it
2800      }
2801    }
2802
2803    private class SortPass {
2804      private int memoryLimit = memory/4;
2805      private int recordLimit = 1000000;
2806      
2807      private DataOutputBuffer rawKeys = new DataOutputBuffer();
2808      private byte[] rawBuffer;
2809
2810      private int[] keyOffsets = new int[1024];
2811      private int[] pointers = new int[keyOffsets.length];
2812      private int[] pointersCopy = new int[keyOffsets.length];
2813      private int[] keyLengths = new int[keyOffsets.length];
2814      private ValueBytes[] rawValues = new ValueBytes[keyOffsets.length];
2815      
2816      private ArrayList segmentLengths = new ArrayList();
2817      
2818      private Reader in = null;
2819      private FSDataOutputStream out = null;
2820      private FSDataOutputStream indexOut = null;
2821      private Path outName;
2822
2823      private Progressable progressable = null;
2824
2825      public int run(boolean deleteInput) throws IOException {
2826        int segments = 0;
2827        int currentFile = 0;
2828        boolean atEof = (currentFile >= inFiles.length);
2829        CompressionType compressionType;
2830        CompressionCodec codec = null;
2831        segmentLengths.clear();
2832        if (atEof) {
2833          return 0;
2834        }
2835        
2836        // Initialize
2837        in = new Reader(fs, inFiles[currentFile], conf);
2838        compressionType = in.getCompressionType();
2839        codec = in.getCompressionCodec();
2840        
2841        for (int i=0; i < rawValues.length; ++i) {
2842          rawValues[i] = null;
2843        }
2844        
2845        while (!atEof) {
2846          int count = 0;
2847          int bytesProcessed = 0;
2848          rawKeys.reset();
2849          while (!atEof && 
2850                 bytesProcessed < memoryLimit && count < recordLimit) {
2851
2852            // Read a record into buffer
2853            // Note: Attempt to re-use 'rawValue' as far as possible
2854            int keyOffset = rawKeys.getLength();       
2855            ValueBytes rawValue = 
2856              (count == keyOffsets.length || rawValues[count] == null) ? 
2857              in.createValueBytes() : 
2858              rawValues[count];
2859            int recordLength = in.nextRaw(rawKeys, rawValue);
2860            if (recordLength == -1) {
2861              in.close();
2862              if (deleteInput) {
2863                fs.delete(inFiles[currentFile], true);
2864              }
2865              currentFile += 1;
2866              atEof = currentFile >= inFiles.length;
2867              if (!atEof) {
2868                in = new Reader(fs, inFiles[currentFile], conf);
2869              } else {
2870                in = null;
2871              }
2872              continue;
2873            }
2874
2875            int keyLength = rawKeys.getLength() - keyOffset;
2876
2877            if (count == keyOffsets.length)
2878              grow();
2879
2880            keyOffsets[count] = keyOffset;                // update pointers
2881            pointers[count] = count;
2882            keyLengths[count] = keyLength;
2883            rawValues[count] = rawValue;
2884
2885            bytesProcessed += recordLength; 
2886            count++;
2887          }
2888
2889          // buffer is full -- sort & flush it
2890          if(LOG.isDebugEnabled()) {
2891            LOG.debug("flushing segment " + segments);
2892          }
2893          rawBuffer = rawKeys.getData();
2894          sort(count);
2895          // indicate we're making progress
2896          if (progressable != null) {
2897            progressable.progress();
2898          }
2899          flush(count, bytesProcessed, compressionType, codec, 
2900                segments==0 && atEof);
2901          segments++;
2902        }
2903        return segments;
2904      }
2905
2906      public void close() throws IOException {
2907        if (in != null) {
2908          in.close();
2909        }
2910        if (out != null) {
2911          out.close();
2912        }
2913        if (indexOut != null) {
2914          indexOut.close();
2915        }
2916      }
2917
2918      private void grow() {
2919        int newLength = keyOffsets.length * 3 / 2;
2920        keyOffsets = grow(keyOffsets, newLength);
2921        pointers = grow(pointers, newLength);
2922        pointersCopy = new int[newLength];
2923        keyLengths = grow(keyLengths, newLength);
2924        rawValues = grow(rawValues, newLength);
2925      }
2926
2927      private int[] grow(int[] old, int newLength) {
2928        int[] result = new int[newLength];
2929        System.arraycopy(old, 0, result, 0, old.length);
2930        return result;
2931      }
2932      
2933      private ValueBytes[] grow(ValueBytes[] old, int newLength) {
2934        ValueBytes[] result = new ValueBytes[newLength];
2935        System.arraycopy(old, 0, result, 0, old.length);
2936        for (int i=old.length; i < newLength; ++i) {
2937          result[i] = null;
2938        }
2939        return result;
2940      }
2941
2942      private void flush(int count, int bytesProcessed, 
2943                         CompressionType compressionType, 
2944                         CompressionCodec codec, 
2945                         boolean done) throws IOException {
2946        if (out == null) {
2947          outName = done ? outFile : outFile.suffix(".0");
2948          out = fs.create(outName);
2949          if (!done) {
2950            indexOut = fs.create(outName.suffix(".index"));
2951          }
2952        }
2953
2954        long segmentStart = out.getPos();
2955        Writer writer = createWriter(conf, Writer.stream(out), 
2956            Writer.keyClass(keyClass), Writer.valueClass(valClass),
2957            Writer.compression(compressionType, codec),
2958            Writer.metadata(done ? metadata : new Metadata()));
2959        
2960        if (!done) {
2961          writer.sync = null;                     // disable sync on temp files
2962        }
2963
2964        for (int i = 0; i < count; i++) {         // write in sorted order
2965          int p = pointers[i];
2966          writer.appendRaw(rawBuffer, keyOffsets[p], keyLengths[p], rawValues[p]);
2967        }
2968        writer.close();
2969        
2970        if (!done) {
2971          // Save the segment length
2972          WritableUtils.writeVLong(indexOut, segmentStart);
2973          WritableUtils.writeVLong(indexOut, (out.getPos()-segmentStart));
2974          indexOut.flush();
2975        }
2976      }
2977
2978      private void sort(int count) {
2979        System.arraycopy(pointers, 0, pointersCopy, 0, count);
2980        mergeSort.mergeSort(pointersCopy, pointers, 0, count);
2981      }
2982      class SeqFileComparator implements Comparator<IntWritable> {
2983        @Override
2984        public int compare(IntWritable I, IntWritable J) {
2985          return comparator.compare(rawBuffer, keyOffsets[I.get()], 
2986                                    keyLengths[I.get()], rawBuffer, 
2987                                    keyOffsets[J.get()], keyLengths[J.get()]);
2988        }
2989      }
2990      
2991      /** set the progressable object in order to report progress */
2992      public void setProgressable(Progressable progressable)
2993      {
2994        this.progressable = progressable;
2995      }
2996      
2997    } // SequenceFile.Sorter.SortPass
2998
2999    /** The interface to iterate over raw keys/values of SequenceFiles. */
3000    public static interface RawKeyValueIterator {
3001      /** Gets the current raw key
3002       * @return DataOutputBuffer
3003       * @throws IOException
3004       */
3005      DataOutputBuffer getKey() throws IOException; 
3006      /** Gets the current raw value
3007       * @return ValueBytes 
3008       * @throws IOException
3009       */
3010      ValueBytes getValue() throws IOException; 
3011      /** Sets up the current key and value (for getKey and getValue)
3012       * @return true if there exists a key/value, false otherwise 
3013       * @throws IOException
3014       */
3015      boolean next() throws IOException;
3016      /** closes the iterator so that the underlying streams can be closed
3017       * @throws IOException
3018       */
3019      void close() throws IOException;
3020      /** Gets the Progress object; this has a float (0.0 - 1.0) 
3021       * indicating the bytes processed by the iterator so far
3022       */
3023      Progress getProgress();
3024    }    
3025    
3026    /**
3027     * Merges the list of segments of type <code>SegmentDescriptor</code>
3028     * @param segments the list of SegmentDescriptors
3029     * @param tmpDir the directory to write temporary files into
3030     * @return RawKeyValueIterator
3031     * @throws IOException
3032     */
3033    public RawKeyValueIterator merge(List <SegmentDescriptor> segments, 
3034                                     Path tmpDir) 
3035      throws IOException {
3036      // pass in object to report progress, if present
3037      MergeQueue mQueue = new MergeQueue(segments, tmpDir, progressable);
3038      return mQueue.merge();
3039    }
3040
3041    /**
3042     * Merges the contents of files passed in Path[] using a max factor value
3043     * that is already set
3044     * @param inNames the array of path names
3045     * @param deleteInputs true if the input files should be deleted when 
3046     * unnecessary
3047     * @param tmpDir the directory to write temporary files into
3048     * @return RawKeyValueIteratorMergeQueue
3049     * @throws IOException
3050     */
3051    public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs,
3052                                     Path tmpDir) 
3053      throws IOException {
3054      return merge(inNames, deleteInputs, 
3055                   (inNames.length < factor) ? inNames.length : factor,
3056                   tmpDir);
3057    }
3058
3059    /**
3060     * Merges the contents of files passed in Path[]
3061     * @param inNames the array of path names
3062     * @param deleteInputs true if the input files should be deleted when 
3063     * unnecessary
3064     * @param factor the factor that will be used as the maximum merge fan-in
3065     * @param tmpDir the directory to write temporary files into
3066     * @return RawKeyValueIteratorMergeQueue
3067     * @throws IOException
3068     */
3069    public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs,
3070                                     int factor, Path tmpDir) 
3071      throws IOException {
3072      //get the segments from inNames
3073      ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
3074      for (int i = 0; i < inNames.length; i++) {
3075        SegmentDescriptor s = new SegmentDescriptor(0,
3076            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
3077        s.preserveInput(!deleteInputs);
3078        s.doSync();
3079        a.add(s);
3080      }
3081      this.factor = factor;
3082      MergeQueue mQueue = new MergeQueue(a, tmpDir, progressable);
3083      return mQueue.merge();
3084    }
3085
3086    /**
3087     * Merges the contents of files passed in Path[]
3088     * @param inNames the array of path names
3089     * @param tempDir the directory for creating temp files during merge
3090     * @param deleteInputs true if the input files should be deleted when 
3091     * unnecessary
3092     * @return RawKeyValueIteratorMergeQueue
3093     * @throws IOException
3094     */
3095    public RawKeyValueIterator merge(Path [] inNames, Path tempDir, 
3096                                     boolean deleteInputs) 
3097      throws IOException {
3098      //outFile will basically be used as prefix for temp files for the
3099      //intermediate merge outputs           
3100      this.outFile = new Path(tempDir + Path.SEPARATOR + "merged");
3101      //get the segments from inNames
3102      ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
3103      for (int i = 0; i < inNames.length; i++) {
3104        SegmentDescriptor s = new SegmentDescriptor(0,
3105            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
3106        s.preserveInput(!deleteInputs);
3107        s.doSync();
3108        a.add(s);
3109      }
3110      factor = (inNames.length < factor) ? inNames.length : factor;
3111      // pass in object to report progress, if present
3112      MergeQueue mQueue = new MergeQueue(a, tempDir, progressable);
3113      return mQueue.merge();
3114    }
3115
3116    /**
3117     * Clones the attributes (like compression of the input file and creates a 
3118     * corresponding Writer
3119     * @param inputFile the path of the input file whose attributes should be 
3120     * cloned
3121     * @param outputFile the path of the output file 
3122     * @param prog the Progressable to report status during the file write
3123     * @return Writer
3124     * @throws IOException
3125     */
3126    public Writer cloneFileAttributes(Path inputFile, Path outputFile, 
3127                                      Progressable prog) throws IOException {
3128      Reader reader = new Reader(conf,
3129                                 Reader.file(inputFile),
3130                                 new Reader.OnlyHeaderOption());
3131      CompressionType compress = reader.getCompressionType();
3132      CompressionCodec codec = reader.getCompressionCodec();
3133      reader.close();
3134
3135      Writer writer = createWriter(conf, 
3136                                   Writer.file(outputFile), 
3137                                   Writer.keyClass(keyClass), 
3138                                   Writer.valueClass(valClass), 
3139                                   Writer.compression(compress, codec), 
3140                                   Writer.progressable(prog));
3141      return writer;
3142    }
3143
3144    /**
3145     * Writes records from RawKeyValueIterator into a file represented by the 
3146     * passed writer
3147     * @param records the RawKeyValueIterator
3148     * @param writer the Writer created earlier 
3149     * @throws IOException
3150     */
3151    public void writeFile(RawKeyValueIterator records, Writer writer) 
3152      throws IOException {
3153      while(records.next()) {
3154        writer.appendRaw(records.getKey().getData(), 0, 
3155                         records.getKey().getLength(), records.getValue());
3156      }
3157      writer.sync();
3158    }
3159        
3160    /** Merge the provided files.
3161     * @param inFiles the array of input path names
3162     * @param outFile the final output file
3163     * @throws IOException
3164     */
3165    public void merge(Path[] inFiles, Path outFile) throws IOException {
3166      if (fs.exists(outFile)) {
3167        throw new IOException("already exists: " + outFile);
3168      }
3169      RawKeyValueIterator r = merge(inFiles, false, outFile.getParent());
3170      Writer writer = cloneFileAttributes(inFiles[0], outFile, null);
3171      
3172      writeFile(r, writer);
3173
3174      writer.close();
3175    }
3176
3177    /** sort calls this to generate the final merged output */
3178    private int mergePass(Path tmpDir) throws IOException {
3179      if(LOG.isDebugEnabled()) {
3180        LOG.debug("running merge pass");
3181      }
3182      Writer writer = cloneFileAttributes(
3183                                          outFile.suffix(".0"), outFile, null);
3184      RawKeyValueIterator r = merge(outFile.suffix(".0"), 
3185                                    outFile.suffix(".0.index"), tmpDir);
3186      writeFile(r, writer);
3187
3188      writer.close();
3189      return 0;
3190    }
3191
3192    /** Used by mergePass to merge the output of the sort
3193     * @param inName the name of the input file containing sorted segments
3194     * @param indexIn the offsets of the sorted segments
3195     * @param tmpDir the relative directory to store intermediate results in
3196     * @return RawKeyValueIterator
3197     * @throws IOException
3198     */
3199    private RawKeyValueIterator merge(Path inName, Path indexIn, Path tmpDir) 
3200      throws IOException {
3201      //get the segments from indexIn
3202      //we create a SegmentContainer so that we can track segments belonging to
3203      //inName and delete inName as soon as we see that we have looked at all
3204      //the contained segments during the merge process & hence don't need 
3205      //them anymore
3206      SegmentContainer container = new SegmentContainer(inName, indexIn);
3207      MergeQueue mQueue = new MergeQueue(container.getSegmentList(), tmpDir, progressable);
3208      return mQueue.merge();
3209    }
3210    
3211    /** This class implements the core of the merge logic */
3212    private class MergeQueue extends PriorityQueue 
3213      implements RawKeyValueIterator {
3214      private boolean compress;
3215      private boolean blockCompress;
3216      private DataOutputBuffer rawKey = new DataOutputBuffer();
3217      private ValueBytes rawValue;
3218      private long totalBytesProcessed;
3219      private float progPerByte;
3220      private Progress mergeProgress = new Progress();
3221      private Path tmpDir;
3222      private Progressable progress = null; //handle to the progress reporting object
3223      private SegmentDescriptor minSegment;
3224      
3225      //a TreeMap used to store the segments sorted by size (segment offset and
3226      //segment path name is used to break ties between segments of same sizes)
3227      private Map<SegmentDescriptor, Void> sortedSegmentSizes =
3228        new TreeMap<SegmentDescriptor, Void>();
3229            
3230      @SuppressWarnings("unchecked")
3231      public void put(SegmentDescriptor stream) throws IOException {
3232        if (size() == 0) {
3233          compress = stream.in.isCompressed();
3234          blockCompress = stream.in.isBlockCompressed();
3235        } else if (compress != stream.in.isCompressed() || 
3236                   blockCompress != stream.in.isBlockCompressed()) {
3237          throw new IOException("All merged files must be compressed or not.");
3238        } 
3239        super.put(stream);
3240      }
3241      
3242      /**
3243       * A queue of file segments to merge
3244       * @param segments the file segments to merge
3245       * @param tmpDir a relative local directory to save intermediate files in
3246       * @param progress the reference to the Progressable object
3247       */
3248      public MergeQueue(List <SegmentDescriptor> segments,
3249          Path tmpDir, Progressable progress) {
3250        int size = segments.size();
3251        for (int i = 0; i < size; i++) {
3252          sortedSegmentSizes.put(segments.get(i), null);
3253        }
3254        this.tmpDir = tmpDir;
3255        this.progress = progress;
3256      }
3257      @Override
3258      protected boolean lessThan(Object a, Object b) {
3259        // indicate we're making progress
3260        if (progress != null) {
3261          progress.progress();
3262        }
3263        SegmentDescriptor msa = (SegmentDescriptor)a;
3264        SegmentDescriptor msb = (SegmentDescriptor)b;
3265        return comparator.compare(msa.getKey().getData(), 0, 
3266                                  msa.getKey().getLength(), msb.getKey().getData(), 0, 
3267                                  msb.getKey().getLength()) < 0;
3268      }
3269      @Override
3270      public void close() throws IOException {
3271        SegmentDescriptor ms;                           // close inputs
3272        while ((ms = (SegmentDescriptor)pop()) != null) {
3273          ms.cleanup();
3274        }
3275        minSegment = null;
3276      }
3277      @Override
3278      public DataOutputBuffer getKey() throws IOException {
3279        return rawKey;
3280      }
3281      @Override
3282      public ValueBytes getValue() throws IOException {
3283        return rawValue;
3284      }
3285      @Override
3286      public boolean next() throws IOException {
3287        if (size() == 0)
3288          return false;
3289        if (minSegment != null) {
3290          //minSegment is non-null for all invocations of next except the first
3291          //one. For the first invocation, the priority queue is ready for use
3292          //but for the subsequent invocations, first adjust the queue 
3293          adjustPriorityQueue(minSegment);
3294          if (size() == 0) {
3295            minSegment = null;
3296            return false;
3297          }
3298        }
3299        minSegment = (SegmentDescriptor)top();
3300        long startPos = minSegment.in.getPosition(); // Current position in stream
3301        //save the raw key reference
3302        rawKey = minSegment.getKey();
3303        //load the raw value. Re-use the existing rawValue buffer
3304        if (rawValue == null) {
3305          rawValue = minSegment.in.createValueBytes();
3306        }
3307        minSegment.nextRawValue(rawValue);
3308        long endPos = minSegment.in.getPosition(); // End position after reading value
3309        updateProgress(endPos - startPos);
3310        return true;
3311      }
3312      
3313      @Override
3314      public Progress getProgress() {
3315        return mergeProgress; 
3316      }
3317
3318      private void adjustPriorityQueue(SegmentDescriptor ms) throws IOException{
3319        long startPos = ms.in.getPosition(); // Current position in stream
3320        boolean hasNext = ms.nextRawKey();
3321        long endPos = ms.in.getPosition(); // End position after reading key
3322        updateProgress(endPos - startPos);
3323        if (hasNext) {
3324          adjustTop();
3325        } else {
3326          pop();
3327          ms.cleanup();
3328        }
3329      }
3330
3331      private void updateProgress(long bytesProcessed) {
3332        totalBytesProcessed += bytesProcessed;
3333        if (progPerByte > 0) {
3334          mergeProgress.set(totalBytesProcessed * progPerByte);
3335        }
3336      }
3337      
3338      /** This is the single level merge that is called multiple times 
3339       * depending on the factor size and the number of segments
3340       * @return RawKeyValueIterator
3341       * @throws IOException
3342       */
3343      public RawKeyValueIterator merge() throws IOException {
3344        //create the MergeStreams from the sorted map created in the constructor
3345        //and dump the final output to a file
3346        int numSegments = sortedSegmentSizes.size();
3347        int origFactor = factor;
3348        int passNo = 1;
3349        LocalDirAllocator lDirAlloc = new LocalDirAllocator("io.seqfile.local.dir");
3350        do {
3351          //get the factor for this pass of merge
3352          factor = getPassFactor(passNo, numSegments);
3353          List<SegmentDescriptor> segmentsToMerge =
3354            new ArrayList<SegmentDescriptor>();
3355          int segmentsConsidered = 0;
3356          int numSegmentsToConsider = factor;
3357          while (true) {
3358            //extract the smallest 'factor' number of segment pointers from the 
3359            //TreeMap. Call cleanup on the empty segments (no key/value data)
3360            SegmentDescriptor[] mStream = 
3361              getSegmentDescriptors(numSegmentsToConsider);
3362            for (int i = 0; i < mStream.length; i++) {
3363              if (mStream[i].nextRawKey()) {
3364                segmentsToMerge.add(mStream[i]);
3365                segmentsConsidered++;
3366                // Count the fact that we read some bytes in calling nextRawKey()
3367                updateProgress(mStream[i].in.getPosition());
3368              }
3369              else {
3370                mStream[i].cleanup();
3371                numSegments--; //we ignore this segment for the merge
3372              }
3373            }
3374            //if we have the desired number of segments
3375            //or looked at all available segments, we break
3376            if (segmentsConsidered == factor || 
3377                sortedSegmentSizes.size() == 0) {
3378              break;
3379            }
3380              
3381            numSegmentsToConsider = factor - segmentsConsidered;
3382          }
3383          //feed the streams to the priority queue
3384          initialize(segmentsToMerge.size()); clear();
3385          for (int i = 0; i < segmentsToMerge.size(); i++) {
3386            put(segmentsToMerge.get(i));
3387          }
3388          //if we have lesser number of segments remaining, then just return the
3389          //iterator, else do another single level merge
3390          if (numSegments <= factor) {
3391            //calculate the length of the remaining segments. Required for 
3392            //calculating the merge progress
3393            long totalBytes = 0;
3394            for (int i = 0; i < segmentsToMerge.size(); i++) {
3395              totalBytes += segmentsToMerge.get(i).segmentLength;
3396            }
3397            if (totalBytes != 0) //being paranoid
3398              progPerByte = 1.0f / (float)totalBytes;
3399            //reset factor to what it originally was
3400            factor = origFactor;
3401            return this;
3402          } else {
3403            //we want to spread the creation of temp files on multiple disks if 
3404            //available under the space constraints
3405            long approxOutputSize = 0; 
3406            for (SegmentDescriptor s : segmentsToMerge) {
3407              approxOutputSize += s.segmentLength + 
3408                                  ChecksumFileSystem.getApproxChkSumLength(
3409                                  s.segmentLength);
3410            }
3411            Path tmpFilename = 
3412              new Path(tmpDir, "intermediate").suffix("." + passNo);
3413
3414            Path outputFile =  lDirAlloc.getLocalPathForWrite(
3415                                                tmpFilename.toString(),
3416                                                approxOutputSize, conf);
3417            if(LOG.isDebugEnabled()) { 
3418              LOG.debug("writing intermediate results to " + outputFile);
3419            }
3420            Writer writer = cloneFileAttributes(
3421                                                fs.makeQualified(segmentsToMerge.get(0).segmentPathName), 
3422                                                fs.makeQualified(outputFile), null);
3423            writer.sync = null; //disable sync for temp files
3424            writeFile(this, writer);
3425            writer.close();
3426            
3427            //we finished one single level merge; now clean up the priority 
3428            //queue
3429            this.close();
3430            
3431            SegmentDescriptor tempSegment = 
3432              new SegmentDescriptor(0,
3433                  fs.getFileStatus(outputFile).getLen(), outputFile);
3434            //put the segment back in the TreeMap
3435            sortedSegmentSizes.put(tempSegment, null);
3436            numSegments = sortedSegmentSizes.size();
3437            passNo++;
3438          }
3439          //we are worried about only the first pass merge factor. So reset the 
3440          //factor to what it originally was
3441          factor = origFactor;
3442        } while(true);
3443      }
3444  
3445      //Hadoop-591
3446      public int getPassFactor(int passNo, int numSegments) {
3447        if (passNo > 1 || numSegments <= factor || factor == 1) 
3448          return factor;
3449        int mod = (numSegments - 1) % (factor - 1);
3450        if (mod == 0)
3451          return factor;
3452        return mod + 1;
3453      }
3454      
3455      /** Return (& remove) the requested number of segment descriptors from the
3456       * sorted map.
3457       */
3458      public SegmentDescriptor[] getSegmentDescriptors(int numDescriptors) {
3459        if (numDescriptors > sortedSegmentSizes.size())
3460          numDescriptors = sortedSegmentSizes.size();
3461        SegmentDescriptor[] SegmentDescriptors = 
3462          new SegmentDescriptor[numDescriptors];
3463        Iterator iter = sortedSegmentSizes.keySet().iterator();
3464        int i = 0;
3465        while (i < numDescriptors) {
3466          SegmentDescriptors[i++] = (SegmentDescriptor)iter.next();
3467          iter.remove();
3468        }
3469        return SegmentDescriptors;
3470      }
3471    } // SequenceFile.Sorter.MergeQueue
3472
3473    /** This class defines a merge segment. This class can be subclassed to 
3474     * provide a customized cleanup method implementation. In this 
3475     * implementation, cleanup closes the file handle and deletes the file 
3476     */
3477    public class SegmentDescriptor implements Comparable {
3478      
3479      long segmentOffset; //the start of the segment in the file
3480      long segmentLength; //the length of the segment
3481      Path segmentPathName; //the path name of the file containing the segment
3482      boolean ignoreSync = true; //set to true for temp files
3483      private Reader in = null; 
3484      private DataOutputBuffer rawKey = null; //this will hold the current key
3485      private boolean preserveInput = false; //delete input segment files?
3486      
3487      /** Constructs a segment
3488       * @param segmentOffset the offset of the segment in the file
3489       * @param segmentLength the length of the segment
3490       * @param segmentPathName the path name of the file containing the segment
3491       */
3492      public SegmentDescriptor (long segmentOffset, long segmentLength, 
3493                                Path segmentPathName) {
3494        this.segmentOffset = segmentOffset;
3495        this.segmentLength = segmentLength;
3496        this.segmentPathName = segmentPathName;
3497      }
3498      
3499      /** Do the sync checks */
3500      public void doSync() {ignoreSync = false;}
3501      
3502      /** Whether to delete the files when no longer needed */
3503      public void preserveInput(boolean preserve) {
3504        preserveInput = preserve;
3505      }
3506
3507      public boolean shouldPreserveInput() {
3508        return preserveInput;
3509      }
3510      
3511      @Override
3512      public int compareTo(Object o) {
3513        SegmentDescriptor that = (SegmentDescriptor)o;
3514        if (this.segmentLength != that.segmentLength) {
3515          return (this.segmentLength < that.segmentLength ? -1 : 1);
3516        }
3517        if (this.segmentOffset != that.segmentOffset) {
3518          return (this.segmentOffset < that.segmentOffset ? -1 : 1);
3519        }
3520        return (this.segmentPathName.toString()).
3521          compareTo(that.segmentPathName.toString());
3522      }
3523
3524      @Override
3525      public boolean equals(Object o) {
3526        if (!(o instanceof SegmentDescriptor)) {
3527          return false;
3528        }
3529        SegmentDescriptor that = (SegmentDescriptor)o;
3530        if (this.segmentLength == that.segmentLength &&
3531            this.segmentOffset == that.segmentOffset &&
3532            this.segmentPathName.toString().equals(
3533              that.segmentPathName.toString())) {
3534          return true;
3535        }
3536        return false;
3537      }
3538
3539      @Override
3540      public int hashCode() {
3541        return 37 * 17 + (int) (segmentOffset^(segmentOffset>>>32));
3542      }
3543
3544      /** Fills up the rawKey object with the key returned by the Reader
3545       * @return true if there is a key returned; false, otherwise
3546       * @throws IOException
3547       */
3548      public boolean nextRawKey() throws IOException {
3549        if (in == null) {
3550          int bufferSize = getBufferSize(conf); 
3551          Reader reader = new Reader(conf,
3552                                     Reader.file(segmentPathName), 
3553                                     Reader.bufferSize(bufferSize),
3554                                     Reader.start(segmentOffset), 
3555                                     Reader.length(segmentLength));
3556        
3557          //sometimes we ignore syncs especially for temp merge files
3558          if (ignoreSync) reader.ignoreSync();
3559
3560          if (reader.getKeyClass() != keyClass)
3561            throw new IOException("wrong key class: " + reader.getKeyClass() +
3562                                  " is not " + keyClass);
3563          if (reader.getValueClass() != valClass)
3564            throw new IOException("wrong value class: "+reader.getValueClass()+
3565                                  " is not " + valClass);
3566          this.in = reader;
3567          rawKey = new DataOutputBuffer();
3568        }
3569        rawKey.reset();
3570        int keyLength = 
3571          in.nextRawKey(rawKey);
3572        return (keyLength >= 0);
3573      }
3574
3575      /** Fills up the passed rawValue with the value corresponding to the key
3576       * read earlier
3577       * @param rawValue
3578       * @return the length of the value
3579       * @throws IOException
3580       */
3581      public int nextRawValue(ValueBytes rawValue) throws IOException {
3582        int valLength = in.nextRawValue(rawValue);
3583        return valLength;
3584      }
3585      
3586      /** Returns the stored rawKey */
3587      public DataOutputBuffer getKey() {
3588        return rawKey;
3589      }
3590      
3591      /** closes the underlying reader */
3592      private void close() throws IOException {
3593        this.in.close();
3594        this.in = null;
3595      }
3596
3597      /** The default cleanup. Subclasses can override this with a custom 
3598       * cleanup 
3599       */
3600      public void cleanup() throws IOException {
3601        close();
3602        if (!preserveInput) {
3603          fs.delete(segmentPathName, true);
3604        }
3605      }
3606    } // SequenceFile.Sorter.SegmentDescriptor
3607    
3608    /** This class provisions multiple segments contained within a single
3609     *  file
3610     */
3611    private class LinkedSegmentsDescriptor extends SegmentDescriptor {
3612
3613      SegmentContainer parentContainer = null;
3614
3615      /** Constructs a segment
3616       * @param segmentOffset the offset of the segment in the file
3617       * @param segmentLength the length of the segment
3618       * @param segmentPathName the path name of the file containing the segment
3619       * @param parent the parent SegmentContainer that holds the segment
3620       */
3621      public LinkedSegmentsDescriptor (long segmentOffset, long segmentLength, 
3622                                       Path segmentPathName, SegmentContainer parent) {
3623        super(segmentOffset, segmentLength, segmentPathName);
3624        this.parentContainer = parent;
3625      }
3626      /** The default cleanup. Subclasses can override this with a custom 
3627       * cleanup 
3628       */
3629      @Override
3630      public void cleanup() throws IOException {
3631        super.close();
3632        if (super.shouldPreserveInput()) return;
3633        parentContainer.cleanup();
3634      }
3635      
3636      @Override
3637      public boolean equals(Object o) {
3638        if (!(o instanceof LinkedSegmentsDescriptor)) {
3639          return false;
3640        }
3641        return super.equals(o);
3642      }
3643    } //SequenceFile.Sorter.LinkedSegmentsDescriptor
3644
3645    /** The class that defines a container for segments to be merged. Primarily
3646     * required to delete temp files as soon as all the contained segments
3647     * have been looked at */
3648    private class SegmentContainer {
3649      private int numSegmentsCleanedUp = 0; //track the no. of segment cleanups
3650      private int numSegmentsContained; //# of segments contained
3651      private Path inName; //input file from where segments are created
3652      
3653      //the list of segments read from the file
3654      private ArrayList <SegmentDescriptor> segments = 
3655        new ArrayList <SegmentDescriptor>();
3656      /** This constructor is there primarily to serve the sort routine that 
3657       * generates a single output file with an associated index file */
3658      public SegmentContainer(Path inName, Path indexIn) throws IOException {
3659        //get the segments from indexIn
3660        FSDataInputStream fsIndexIn = fs.open(indexIn);
3661        long end = fs.getFileStatus(indexIn).getLen();
3662        while (fsIndexIn.getPos() < end) {
3663          long segmentOffset = WritableUtils.readVLong(fsIndexIn);
3664          long segmentLength = WritableUtils.readVLong(fsIndexIn);
3665          Path segmentName = inName;
3666          segments.add(new LinkedSegmentsDescriptor(segmentOffset, 
3667                                                    segmentLength, segmentName, this));
3668        }
3669        fsIndexIn.close();
3670        fs.delete(indexIn, true);
3671        numSegmentsContained = segments.size();
3672        this.inName = inName;
3673      }
3674
3675      public List <SegmentDescriptor> getSegmentList() {
3676        return segments;
3677      }
3678      public void cleanup() throws IOException {
3679        numSegmentsCleanedUp++;
3680        if (numSegmentsCleanedUp == numSegmentsContained) {
3681          fs.delete(inName, true);
3682        }
3683      }
3684    } //SequenceFile.Sorter.SegmentContainer
3685
3686  } // SequenceFile.Sorter
3687
3688} // SequenceFile