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