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