1 /*
2  *  Licensed to the Apache Software Foundation (ASF) under one or more
3  *  contributor license agreements.  See the NOTICE file distributed with
4  *  this work for additional information regarding copyright ownership.
5  *  The ASF licenses this file to You under the Apache License, Version 2.0
6  *  (the "License"); you may not use this file except in compliance with
7  *  the License.  You may obtain a copy of the License at
8  *
9  *      http://www.apache.org/licenses/LICENSE-2.0
10  *
11  *  Unless required by applicable law or agreed to in writing, software
12  *  distributed under the License is distributed on an "AS IS" BASIS,
13  *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14  *  See the License for the specific language governing permissions and
15  *  limitations under the License.
16  *
17  */
18 
19 /**
20  * This package is based on the work done by Keiron Liddle, Aftex Software
21  * <keiron@aftexsw.com> to whom the Ant project is very grateful for his
22  * great code.
23  * <p>
24  * This code was enhanced so that it can de-compress blocks of bzip2 data.
25  * Current position in the stream is an important statistic for Hadoop. For
26  * example in LineRecordReader, we solely depend on the current position in the
27  * stream to know about the progess. The notion of position becomes complicated
28  * for compressed files. The Hadoop splitting is done in terms of compressed
29  * file. But a compressed file deflates to a large amount of data. So we have
30  * handled this problem in the following way.
31  *
32  * On object creation time, we find the next block start delimiter. Once such a
33  * marker is found, the stream stops there (we discard any read compressed data
34  * in this process) and the position is updated (i.e. the caller of this class
35  * will find out the stream location). At this point we are ready for actual
36  * reading (i.e. decompression) of data.
37  *
38  * The subsequent read calls give out data. The position is updated when the
39  * caller of this class has read off the current block + 1 bytes. In between the
40  * block reading, position is not updated. (We can only update the postion on
41  * block boundaries).
42  */
43 package org.apache.hadoop.io.compress.bzip2;
44 
45 import java.io.BufferedInputStream;
46 import java.io.InputStream;
47 import java.io.IOException;
48 
49 import org.apache.hadoop.io.compress.SplittableCompressionCodec.READ_MODE;
50 
51 
52 /**
53  * An input stream that decompresses from the BZip2 format (without the file
54  * header chars) to be read as any other stream.
55  *
56  * <p>
57  * The decompression requires large amounts of memory. Thus you should call the
58  * {@link #close() close()} method as soon as possible, to force
59  * <tt>CBZip2InputStream</tt> to release the allocated memory. See
60  * {@link CBZip2OutputStream CBZip2OutputStream} for information about memory
61  * usage.
62  * </p>
63  *
64  * <p>
65  * <tt>CBZip2InputStream</tt> reads bytes from the compressed source stream via
66  * the single byte {@link java.io.InputStream#read() read()} method exclusively.
67  * Thus you should consider to use a buffered source stream.
68  * </p>
69  *
70  * <p>
71  * Instances of this class are not threadsafe.
72  * </p>
73  */
74 public class CBZip2InputStream extends InputStream implements BZip2Constants {
75 
76   public static final long BLOCK_DELIMITER = 0X314159265359L;// start of block
77   public static final long EOS_DELIMITER = 0X177245385090L;// end of bzip2 stream
78   private static final int DELIMITER_BIT_LENGTH = 48;
79   READ_MODE readMode = READ_MODE.CONTINUOUS;
80   // The variable records the current advertised position of the stream.
81   private long reportedBytesReadFromCompressedStream = 0L;
82   // The following variable keep record of compressed bytes read.
83   private long bytesReadFromCompressedStream = 0L;
84   private boolean lazyInitialization = false;
85   private byte array[] = new byte[1];
86 
87   /**
88   * Index of the last char in the block, so the block size == last + 1.
89   */
90   private int last;
91 
92   /**
93   * Index in zptr[] of original string after sorting.
94   */
95   private int origPtr;
96 
97   /**
98   * always: in the range 0 .. 9. The current block size is 100000 * this
99   * number.
100   */
101   private int blockSize100k;
102 
103   private boolean blockRandomised = false;
104 
105   private long bsBuff;
106   private long bsLive;
107   private final CRC crc = new CRC();
108 
109   private int nInUse;
110 
111   private BufferedInputStream in;
112 
113   private int currentChar = -1;
114 
115   /**
116    * A state machine to keep track of current state of the de-coder
117    *
118    */
119   public enum STATE {
120     EOF, START_BLOCK_STATE, RAND_PART_A_STATE, RAND_PART_B_STATE, RAND_PART_C_STATE, NO_RAND_PART_A_STATE, NO_RAND_PART_B_STATE, NO_RAND_PART_C_STATE, NO_PROCESS_STATE
121   };
122 
123   private STATE currentState = STATE.START_BLOCK_STATE;
124 
125   private int storedBlockCRC, storedCombinedCRC;
126   private int computedBlockCRC, computedCombinedCRC;
127 
128   private boolean skipResult = false;// used by skipToNextMarker
129   private static boolean skipDecompression = false;
130 
131   // Variables used by setup* methods exclusively
132 
133   private int su_count;
134   private int su_ch2;
135   private int su_chPrev;
136   private int su_i2;
137   private int su_j2;
138   private int su_rNToGo;
139   private int su_rTPos;
140   private int su_tPos;
141   private char su_z;
142 
143   /**
144   * All memory intensive stuff. This field is initialized by initBlock().
145   */
146   private CBZip2InputStream.Data data;
147 
148   /**
149   * This method reports the processed bytes so far. Please note that this
150   * statistic is only updated on block boundaries and only when the stream is
151   * initiated in BYBLOCK mode.
152   */
getProcessedByteCount()153   public long getProcessedByteCount() {
154     return reportedBytesReadFromCompressedStream;
155   }
156 
157   /**
158    * This method keeps track of raw processed compressed
159    * bytes.
160    *
161    * @param count count is the number of bytes to be
162    *           added to raw processed bytes
163    */
164 
updateProcessedByteCount(int count)165   protected void updateProcessedByteCount(int count) {
166     this.bytesReadFromCompressedStream += count;
167   }
168 
169   /**
170    * This method is called by the client of this
171    * class in case there are any corrections in
172    * the stream position.  One common example is
173    * when client of this code removes starting BZ
174    * characters from the compressed stream.
175    *
176    * @param count count bytes are added to the reported bytes
177    *
178    */
updateReportedByteCount(int count)179   public void updateReportedByteCount(int count) {
180     this.reportedBytesReadFromCompressedStream += count;
181     this.updateProcessedByteCount(count);
182   }
183 
184   /**
185   * This method reads a Byte from the compressed stream. Whenever we need to
186   * read from the underlying compressed stream, this method should be called
187   * instead of directly calling the read method of the underlying compressed
188   * stream. This method does important record keeping to have the statistic
189   * that how many bytes have been read off the compressed stream.
190   */
readAByte(InputStream inStream)191   private int readAByte(InputStream inStream) throws IOException {
192     int read = inStream.read();
193     if (read >= 0) {
194       this.updateProcessedByteCount(1);
195     }
196     return read;
197   }
198 
199   /**
200   * This method tries to find the marker (passed to it as the first parameter)
201   * in the stream.  It can find bit patterns of length <= 63 bits.  Specifically
202   * this method is used in CBZip2InputStream to find the end of block (EOB)
203   * delimiter in the stream, starting from the current position of the stream.
204   * If marker is found, the stream position will be right after marker at the
205   * end of this call.
206   *
207   * @param marker  The bit pattern to be found in the stream
208   * @param markerBitLength  No of bits in the marker
209   *
210   * @throws IOException
211   * @throws IllegalArgumentException  if marketBitLength is greater than 63
212   */
skipToNextMarker(long marker, int markerBitLength)213   public boolean skipToNextMarker(long marker, int markerBitLength)
214       throws IOException, IllegalArgumentException {
215     try {
216       if (markerBitLength > 63) {
217         throw new IllegalArgumentException(
218             "skipToNextMarker can not find patterns greater than 63 bits");
219       }
220       // pick next marketBitLength bits in the stream
221       long bytes = 0;
222       bytes = this.bsR(markerBitLength);
223       if (bytes == -1) {
224         return false;
225       }
226       while (true) {
227         if (bytes == marker) {
228           return true;
229 
230         } else {
231           bytes = bytes << 1;
232           bytes = bytes & ((1L << markerBitLength) - 1);
233           int oneBit = (int) this.bsR(1);
234           if (oneBit != -1) {
235             bytes = bytes | oneBit;
236           } else
237             return false;
238         }
239       }
240     } catch (IOException ex) {
241       return false;
242     }
243   }
244 
reportCRCError()245   protected void reportCRCError() throws IOException {
246     throw new IOException("crc error");
247   }
248 
makeMaps()249   private void makeMaps() {
250     final boolean[] inUse = this.data.inUse;
251     final byte[] seqToUnseq = this.data.seqToUnseq;
252 
253     int nInUseShadow = 0;
254 
255     for (int i = 0; i < 256; i++) {
256       if (inUse[i])
257         seqToUnseq[nInUseShadow++] = (byte) i;
258     }
259 
260     this.nInUse = nInUseShadow;
261   }
262 
263   /**
264   * Constructs a new CBZip2InputStream which decompresses bytes read from the
265   * specified stream.
266   *
267   * <p>
268   * Although BZip2 headers are marked with the magic <tt>"Bz"</tt> this
269   * constructor expects the next byte in the stream to be the first one after
270   * the magic. Thus callers have to skip the first two bytes. Otherwise this
271   * constructor will throw an exception.
272   * </p>
273   *
274   * @throws IOException
275   *             if the stream content is malformed or an I/O error occurs.
276   * @throws NullPointerException
277   *             if <tt>in == null</tt>
278   */
CBZip2InputStream(final InputStream in, READ_MODE readMode)279   public CBZip2InputStream(final InputStream in, READ_MODE readMode)
280       throws IOException {
281 
282     super();
283     int blockSize = 0X39;// i.e 9
284     this.blockSize100k = blockSize - '0';
285     this.in = new BufferedInputStream(in, 1024 * 9);// >1 MB buffer
286     this.readMode = readMode;
287     if (readMode == READ_MODE.CONTINUOUS) {
288       currentState = STATE.START_BLOCK_STATE;
289       lazyInitialization = (in.available() == 0)?true:false;
290       if(!lazyInitialization){
291     init();
292   }
293     } else if (readMode == READ_MODE.BYBLOCK) {
294       this.currentState = STATE.NO_PROCESS_STATE;
295       skipResult = this.skipToNextMarker(CBZip2InputStream.BLOCK_DELIMITER,DELIMITER_BIT_LENGTH);
296       this.reportedBytesReadFromCompressedStream = this.bytesReadFromCompressedStream;
297       if(!skipDecompression){
298         changeStateToProcessABlock();
299       }
300     }
301   }
302 
303   /**
304    * Returns the number of bytes between the current stream position
305    * and the immediate next BZip2 block marker.
306    *
307    * @param in
308    *             The InputStream
309    *
310    * @return long Number of bytes between current stream position and the
311    * next BZip2 block start marker.
312  * @throws IOException
313    *
314    */
numberOfBytesTillNextMarker(final InputStream in)315   public static long numberOfBytesTillNextMarker(final InputStream in) throws IOException{
316     CBZip2InputStream.skipDecompression = true;
317     CBZip2InputStream anObject = null;
318 
319     anObject = new CBZip2InputStream(in, READ_MODE.BYBLOCK);
320 
321     return anObject.getProcessedByteCount();
322   }
323 
CBZip2InputStream(final InputStream in)324   public CBZip2InputStream(final InputStream in) throws IOException {
325     this(in, READ_MODE.CONTINUOUS);
326   }
327 
changeStateToProcessABlock()328   private void changeStateToProcessABlock() throws IOException {
329     if (skipResult == true) {
330       initBlock();
331       setupBlock();
332     } else {
333       this.currentState = STATE.EOF;
334     }
335   }
336 
337 
read()338   public int read() throws IOException {
339 
340     if (this.in != null) {
341       int result = this.read(array, 0, 1);
342       int value = 0XFF & array[0];
343       return (result > 0 ? value : result);
344 
345     } else {
346       throw new IOException("stream closed");
347     }
348   }
349 
350   /**
351    * In CONTINOUS reading mode, this read method starts from the
352    * start of the compressed stream and end at the end of file by
353    * emitting un-compressed data.  In this mode stream positioning
354    * is not announced and should be ignored.
355    *
356    * In BYBLOCK reading mode, this read method informs about the end
357    * of a BZip2 block by returning EOB.  At this event, the compressed
358    * stream position is also announced.  This announcement tells that
359    * how much of the compressed stream has been de-compressed and read
360    * out of this class.  In between EOB events, the stream position is
361    * not updated.
362    *
363    *
364    * @throws IOException
365    *             if the stream content is malformed or an I/O error occurs.
366    *
367    * @return int The return value greater than 0 are the bytes read.  A value
368    * of -1 means end of stream while -2 represents end of block
369    */
370 
371 
read(final byte[] dest, final int offs, final int len)372   public int read(final byte[] dest, final int offs, final int len)
373       throws IOException {
374     if (offs < 0) {
375       throw new IndexOutOfBoundsException("offs(" + offs + ") < 0.");
376     }
377     if (len < 0) {
378       throw new IndexOutOfBoundsException("len(" + len + ") < 0.");
379     }
380     if (offs + len > dest.length) {
381       throw new IndexOutOfBoundsException("offs(" + offs + ") + len("
382           + len + ") > dest.length(" + dest.length + ").");
383     }
384     if (this.in == null) {
385       throw new IOException("stream closed");
386     }
387 
388     if(lazyInitialization){
389       this.init();
390       this.lazyInitialization = false;
391     }
392 
393     if(skipDecompression){
394       changeStateToProcessABlock();
395       CBZip2InputStream.skipDecompression = false;
396     }
397 
398     final int hi = offs + len;
399     int destOffs = offs;
400     int b = 0;
401 
402 
403 
404     for (; ((destOffs < hi) && ((b = read0())) >= 0);) {
405       dest[destOffs++] = (byte) b;
406 
407     }
408 
409     int result = destOffs - offs;
410     if (result == 0) {
411       //report 'end of block' or 'end of stream'
412       result = b;
413 
414       skipResult = this.skipToNextMarker(CBZip2InputStream.BLOCK_DELIMITER, DELIMITER_BIT_LENGTH);
415       //Exactly when we are about to start a new block, we advertise the stream position.
416       this.reportedBytesReadFromCompressedStream = this.bytesReadFromCompressedStream;
417 
418       changeStateToProcessABlock();
419     }
420     return result;
421   }
422 
read0()423   private int read0() throws IOException {
424     final int retChar = this.currentChar;
425 
426     switch (this.currentState) {
427     case EOF:
428       return END_OF_STREAM;// return -1
429 
430     case NO_PROCESS_STATE:
431       return END_OF_BLOCK;// return -2
432 
433     case START_BLOCK_STATE:
434       throw new IllegalStateException();
435 
436     case RAND_PART_A_STATE:
437       throw new IllegalStateException();
438 
439     case RAND_PART_B_STATE:
440       setupRandPartB();
441       break;
442 
443     case RAND_PART_C_STATE:
444       setupRandPartC();
445       break;
446 
447     case NO_RAND_PART_A_STATE:
448       throw new IllegalStateException();
449 
450     case NO_RAND_PART_B_STATE:
451       setupNoRandPartB();
452       break;
453 
454     case NO_RAND_PART_C_STATE:
455       setupNoRandPartC();
456       break;
457 
458     default:
459       throw new IllegalStateException();
460     }
461 
462     return retChar;
463   }
464 
init()465   private void init() throws IOException {
466     int magic2 = this.readAByte(in);
467     if (magic2 != 'h') {
468       throw new IOException("Stream is not BZip2 formatted: expected 'h'"
469           + " as first byte but got '" + (char) magic2 + "'");
470     }
471 
472     int blockSize = this.readAByte(in);
473     if ((blockSize < '1') || (blockSize > '9')) {
474       throw new IOException("Stream is not BZip2 formatted: illegal "
475           + "blocksize " + (char) blockSize);
476     }
477 
478     this.blockSize100k = blockSize - '0';
479 
480     initBlock();
481     setupBlock();
482   }
483 
initBlock()484   private void initBlock() throws IOException {
485     if (this.readMode == READ_MODE.BYBLOCK) {
486       // this.checkBlockIntegrity();
487       this.storedBlockCRC = bsGetInt();
488       this.blockRandomised = bsR(1) == 1;
489 
490       /**
491       * Allocate data here instead in constructor, so we do not allocate
492       * it if the input file is empty.
493       */
494       if (this.data == null) {
495         this.data = new Data(this.blockSize100k);
496       }
497 
498       // currBlockNo++;
499       getAndMoveToFrontDecode();
500 
501       this.crc.initialiseCRC();
502       this.currentState = STATE.START_BLOCK_STATE;
503       return;
504     }
505 
506     char magic0 = bsGetUByte();
507     char magic1 = bsGetUByte();
508     char magic2 = bsGetUByte();
509     char magic3 = bsGetUByte();
510     char magic4 = bsGetUByte();
511     char magic5 = bsGetUByte();
512 
513     if (magic0 == 0x17 && magic1 == 0x72 && magic2 == 0x45
514         && magic3 == 0x38 && magic4 == 0x50 && magic5 == 0x90) {
515       complete(); // end of file
516     } else if (magic0 != 0x31 || // '1'
517         magic1 != 0x41 || // ')'
518         magic2 != 0x59 || // 'Y'
519         magic3 != 0x26 || // '&'
520         magic4 != 0x53 || // 'S'
521         magic5 != 0x59 // 'Y'
522     ) {
523       this.currentState = STATE.EOF;
524       throw new IOException("bad block header");
525     } else {
526       this.storedBlockCRC = bsGetInt();
527       this.blockRandomised = bsR(1) == 1;
528 
529       /**
530       * Allocate data here instead in constructor, so we do not allocate
531       * it if the input file is empty.
532       */
533       if (this.data == null) {
534         this.data = new Data(this.blockSize100k);
535       }
536 
537       // currBlockNo++;
538       getAndMoveToFrontDecode();
539 
540       this.crc.initialiseCRC();
541       this.currentState = STATE.START_BLOCK_STATE;
542     }
543   }
544 
endBlock()545   private void endBlock() throws IOException {
546     this.computedBlockCRC = this.crc.getFinalCRC();
547 
548     // A bad CRC is considered a fatal error.
549     if (this.storedBlockCRC != this.computedBlockCRC) {
550       // make next blocks readable without error
551       // (repair feature, not yet documented, not tested)
552       this.computedCombinedCRC = (this.storedCombinedCRC << 1)
553           | (this.storedCombinedCRC >>> 31);
554       this.computedCombinedCRC ^= this.storedBlockCRC;
555 
556       reportCRCError();
557     }
558 
559     this.computedCombinedCRC = (this.computedCombinedCRC << 1)
560         | (this.computedCombinedCRC >>> 31);
561     this.computedCombinedCRC ^= this.computedBlockCRC;
562   }
563 
complete()564   private void complete() throws IOException {
565     this.storedCombinedCRC = bsGetInt();
566     this.currentState = STATE.EOF;
567     this.data = null;
568 
569     if (this.storedCombinedCRC != this.computedCombinedCRC) {
570       reportCRCError();
571     }
572   }
573 
close()574   public void close() throws IOException {
575     InputStream inShadow = this.in;
576     if (inShadow != null) {
577       try {
578         if (inShadow != System.in) {
579           inShadow.close();
580         }
581       } finally {
582         this.data = null;
583         this.in = null;
584       }
585     }
586   }
587 
bsR(final long n)588   private long bsR(final long n) throws IOException {
589     long bsLiveShadow = this.bsLive;
590     long bsBuffShadow = this.bsBuff;
591 
592     if (bsLiveShadow < n) {
593       final InputStream inShadow = this.in;
594       do {
595         int thech = readAByte(inShadow);
596 
597         if (thech < 0) {
598           throw new IOException("unexpected end of stream");
599         }
600 
601         bsBuffShadow = (bsBuffShadow << 8) | thech;
602         bsLiveShadow += 8;
603       } while (bsLiveShadow < n);
604 
605       this.bsBuff = bsBuffShadow;
606     }
607 
608     this.bsLive = bsLiveShadow - n;
609     return (bsBuffShadow >> (bsLiveShadow - n)) & ((1L << n) - 1);
610   }
611 
bsGetBit()612   private boolean bsGetBit() throws IOException {
613     long bsLiveShadow = this.bsLive;
614     long bsBuffShadow = this.bsBuff;
615 
616     if (bsLiveShadow < 1) {
617       int thech = this.readAByte(in);
618 
619       if (thech < 0) {
620         throw new IOException("unexpected end of stream");
621       }
622 
623       bsBuffShadow = (bsBuffShadow << 8) | thech;
624       bsLiveShadow += 8;
625       this.bsBuff = bsBuffShadow;
626     }
627 
628     this.bsLive = bsLiveShadow - 1;
629     return ((bsBuffShadow >> (bsLiveShadow - 1)) & 1) != 0;
630   }
631 
bsGetUByte()632   private char bsGetUByte() throws IOException {
633     return (char) bsR(8);
634   }
635 
bsGetInt()636   private int bsGetInt() throws IOException {
637     return (int) ((((((bsR(8) << 8) | bsR(8)) << 8) | bsR(8)) << 8) | bsR(8));
638   }
639 
640   /**
641   * Called by createHuffmanDecodingTables() exclusively.
642   */
hbCreateDecodeTables(final int[] limit, final int[] base, final int[] perm, final char[] length, final int minLen, final int maxLen, final int alphaSize)643   private static void hbCreateDecodeTables(final int[] limit,
644       final int[] base, final int[] perm, final char[] length,
645       final int minLen, final int maxLen, final int alphaSize) {
646     for (int i = minLen, pp = 0; i <= maxLen; i++) {
647       for (int j = 0; j < alphaSize; j++) {
648         if (length[j] == i) {
649           perm[pp++] = j;
650         }
651       }
652     }
653 
654     for (int i = MAX_CODE_LEN; --i > 0;) {
655       base[i] = 0;
656       limit[i] = 0;
657     }
658 
659     for (int i = 0; i < alphaSize; i++) {
660       base[length[i] + 1]++;
661     }
662 
663     for (int i = 1, b = base[0]; i < MAX_CODE_LEN; i++) {
664       b += base[i];
665       base[i] = b;
666     }
667 
668     for (int i = minLen, vec = 0, b = base[i]; i <= maxLen; i++) {
669       final int nb = base[i + 1];
670       vec += nb - b;
671       b = nb;
672       limit[i] = vec - 1;
673       vec <<= 1;
674     }
675 
676     for (int i = minLen + 1; i <= maxLen; i++) {
677       base[i] = ((limit[i - 1] + 1) << 1) - base[i];
678     }
679   }
680 
recvDecodingTables()681   private void recvDecodingTables() throws IOException {
682     final Data dataShadow = this.data;
683     final boolean[] inUse = dataShadow.inUse;
684     final byte[] pos = dataShadow.recvDecodingTables_pos;
685     final byte[] selector = dataShadow.selector;
686     final byte[] selectorMtf = dataShadow.selectorMtf;
687 
688     int inUse16 = 0;
689 
690     /* Receive the mapping table */
691     for (int i = 0; i < 16; i++) {
692       if (bsGetBit()) {
693         inUse16 |= 1 << i;
694       }
695     }
696 
697     for (int i = 256; --i >= 0;) {
698       inUse[i] = false;
699     }
700 
701     for (int i = 0; i < 16; i++) {
702       if ((inUse16 & (1 << i)) != 0) {
703         final int i16 = i << 4;
704         for (int j = 0; j < 16; j++) {
705           if (bsGetBit()) {
706             inUse[i16 + j] = true;
707           }
708         }
709       }
710     }
711 
712     makeMaps();
713     final int alphaSize = this.nInUse + 2;
714 
715     /* Now the selectors */
716     final int nGroups = (int) bsR(3);
717     final int nSelectors = (int) bsR(15);
718 
719     for (int i = 0; i < nSelectors; i++) {
720       int j = 0;
721       while (bsGetBit()) {
722         j++;
723       }
724       selectorMtf[i] = (byte) j;
725     }
726 
727     /* Undo the MTF values for the selectors. */
728     for (int v = nGroups; --v >= 0;) {
729       pos[v] = (byte) v;
730     }
731 
732     for (int i = 0; i < nSelectors; i++) {
733       int v = selectorMtf[i] & 0xff;
734       final byte tmp = pos[v];
735       while (v > 0) {
736         // nearly all times v is zero, 4 in most other cases
737         pos[v] = pos[v - 1];
738         v--;
739       }
740       pos[0] = tmp;
741       selector[i] = tmp;
742     }
743 
744     final char[][] len = dataShadow.temp_charArray2d;
745 
746     /* Now the coding tables */
747     for (int t = 0; t < nGroups; t++) {
748       int curr = (int) bsR(5);
749       final char[] len_t = len[t];
750       for (int i = 0; i < alphaSize; i++) {
751         while (bsGetBit()) {
752           curr += bsGetBit() ? -1 : 1;
753         }
754         len_t[i] = (char) curr;
755       }
756     }
757 
758     // finally create the Huffman tables
759     createHuffmanDecodingTables(alphaSize, nGroups);
760   }
761 
762   /**
763   * Called by recvDecodingTables() exclusively.
764   */
createHuffmanDecodingTables(final int alphaSize, final int nGroups)765   private void createHuffmanDecodingTables(final int alphaSize,
766       final int nGroups) {
767     final Data dataShadow = this.data;
768     final char[][] len = dataShadow.temp_charArray2d;
769     final int[] minLens = dataShadow.minLens;
770     final int[][] limit = dataShadow.limit;
771     final int[][] base = dataShadow.base;
772     final int[][] perm = dataShadow.perm;
773 
774     for (int t = 0; t < nGroups; t++) {
775       int minLen = 32;
776       int maxLen = 0;
777       final char[] len_t = len[t];
778       for (int i = alphaSize; --i >= 0;) {
779         final char lent = len_t[i];
780         if (lent > maxLen) {
781           maxLen = lent;
782         }
783         if (lent < minLen) {
784           minLen = lent;
785         }
786       }
787       hbCreateDecodeTables(limit[t], base[t], perm[t], len[t], minLen,
788           maxLen, alphaSize);
789       minLens[t] = minLen;
790     }
791   }
792 
getAndMoveToFrontDecode()793   private void getAndMoveToFrontDecode() throws IOException {
794     this.origPtr = (int) bsR(24);
795     recvDecodingTables();
796 
797     final InputStream inShadow = this.in;
798     final Data dataShadow = this.data;
799     final byte[] ll8 = dataShadow.ll8;
800     final int[] unzftab = dataShadow.unzftab;
801     final byte[] selector = dataShadow.selector;
802     final byte[] seqToUnseq = dataShadow.seqToUnseq;
803     final char[] yy = dataShadow.getAndMoveToFrontDecode_yy;
804     final int[] minLens = dataShadow.minLens;
805     final int[][] limit = dataShadow.limit;
806     final int[][] base = dataShadow.base;
807     final int[][] perm = dataShadow.perm;
808     final int limitLast = this.blockSize100k * 100000;
809 
810     /*
811     * Setting up the unzftab entries here is not strictly necessary, but it
812     * does save having to do it later in a separate pass, and so saves a
813     * block's worth of cache misses.
814     */
815     for (int i = 256; --i >= 0;) {
816       yy[i] = (char) i;
817       unzftab[i] = 0;
818     }
819 
820     int groupNo = 0;
821     int groupPos = G_SIZE - 1;
822     final int eob = this.nInUse + 1;
823     int nextSym = getAndMoveToFrontDecode0(0);
824     int bsBuffShadow = (int) this.bsBuff;
825     int bsLiveShadow = (int) this.bsLive;
826     int lastShadow = -1;
827     int zt = selector[groupNo] & 0xff;
828     int[] base_zt = base[zt];
829     int[] limit_zt = limit[zt];
830     int[] perm_zt = perm[zt];
831     int minLens_zt = minLens[zt];
832 
833     while (nextSym != eob) {
834       if ((nextSym == RUNA) || (nextSym == RUNB)) {
835         int s = -1;
836 
837         for (int n = 1; true; n <<= 1) {
838           if (nextSym == RUNA) {
839             s += n;
840           } else if (nextSym == RUNB) {
841             s += n << 1;
842           } else {
843             break;
844           }
845 
846           if (groupPos == 0) {
847             groupPos = G_SIZE - 1;
848             zt = selector[++groupNo] & 0xff;
849             base_zt = base[zt];
850             limit_zt = limit[zt];
851             perm_zt = perm[zt];
852             minLens_zt = minLens[zt];
853           } else {
854             groupPos--;
855           }
856 
857           int zn = minLens_zt;
858 
859           while (bsLiveShadow < zn) {
860             final int thech = readAByte(inShadow);
861             if (thech >= 0) {
862               bsBuffShadow = (bsBuffShadow << 8) | thech;
863               bsLiveShadow += 8;
864               continue;
865             } else {
866               throw new IOException("unexpected end of stream");
867             }
868           }
869           long zvec = (bsBuffShadow >> (bsLiveShadow - zn))
870               & ((1 << zn) - 1);
871           bsLiveShadow -= zn;
872 
873           while (zvec > limit_zt[zn]) {
874             zn++;
875             while (bsLiveShadow < 1) {
876               final int thech = readAByte(inShadow);
877               if (thech >= 0) {
878                 bsBuffShadow = (bsBuffShadow << 8) | thech;
879                 bsLiveShadow += 8;
880                 continue;
881               } else {
882                 throw new IOException(
883                     "unexpected end of stream");
884               }
885             }
886             bsLiveShadow--;
887             zvec = (zvec << 1)
888                 | ((bsBuffShadow >> bsLiveShadow) & 1);
889           }
890           nextSym = perm_zt[(int) (zvec - base_zt[zn])];
891         }
892 
893         final byte ch = seqToUnseq[yy[0]];
894         unzftab[ch & 0xff] += s + 1;
895 
896         while (s-- >= 0) {
897           ll8[++lastShadow] = ch;
898         }
899 
900         if (lastShadow >= limitLast) {
901           throw new IOException("block overrun");
902         }
903       } else {
904         if (++lastShadow >= limitLast) {
905           throw new IOException("block overrun");
906         }
907 
908         final char tmp = yy[nextSym - 1];
909         unzftab[seqToUnseq[tmp] & 0xff]++;
910         ll8[lastShadow] = seqToUnseq[tmp];
911 
912         /*
913         * This loop is hammered during decompression, hence avoid
914         * native method call overhead of System.arraycopy for very
915         * small ranges to copy.
916         */
917         if (nextSym <= 16) {
918           for (int j = nextSym - 1; j > 0;) {
919             yy[j] = yy[--j];
920           }
921         } else {
922           System.arraycopy(yy, 0, yy, 1, nextSym - 1);
923         }
924 
925         yy[0] = tmp;
926 
927         if (groupPos == 0) {
928           groupPos = G_SIZE - 1;
929           zt = selector[++groupNo] & 0xff;
930           base_zt = base[zt];
931           limit_zt = limit[zt];
932           perm_zt = perm[zt];
933           minLens_zt = minLens[zt];
934         } else {
935           groupPos--;
936         }
937 
938         int zn = minLens_zt;
939 
940         while (bsLiveShadow < zn) {
941           final int thech = readAByte(inShadow);
942           if (thech >= 0) {
943             bsBuffShadow = (bsBuffShadow << 8) | thech;
944             bsLiveShadow += 8;
945             continue;
946           } else {
947             throw new IOException("unexpected end of stream");
948           }
949         }
950         int zvec = (bsBuffShadow >> (bsLiveShadow - zn))
951             & ((1 << zn) - 1);
952         bsLiveShadow -= zn;
953 
954         while (zvec > limit_zt[zn]) {
955           zn++;
956           while (bsLiveShadow < 1) {
957             final int thech = readAByte(inShadow);
958             if (thech >= 0) {
959               bsBuffShadow = (bsBuffShadow << 8) | thech;
960               bsLiveShadow += 8;
961               continue;
962             } else {
963               throw new IOException("unexpected end of stream");
964             }
965           }
966           bsLiveShadow--;
967           zvec = ((zvec << 1) | ((bsBuffShadow >> bsLiveShadow) & 1));
968         }
969         nextSym = perm_zt[zvec - base_zt[zn]];
970       }
971     }
972 
973     this.last = lastShadow;
974     this.bsLive = bsLiveShadow;
975     this.bsBuff = bsBuffShadow;
976   }
977 
getAndMoveToFrontDecode0(final int groupNo)978   private int getAndMoveToFrontDecode0(final int groupNo) throws IOException {
979     final InputStream inShadow = this.in;
980     final Data dataShadow = this.data;
981     final int zt = dataShadow.selector[groupNo] & 0xff;
982     final int[] limit_zt = dataShadow.limit[zt];
983     int zn = dataShadow.minLens[zt];
984     int zvec = (int) bsR(zn);
985     int bsLiveShadow = (int) this.bsLive;
986     int bsBuffShadow = (int) this.bsBuff;
987 
988     while (zvec > limit_zt[zn]) {
989       zn++;
990       while (bsLiveShadow < 1) {
991         final int thech = readAByte(inShadow);
992 
993         if (thech >= 0) {
994           bsBuffShadow = (bsBuffShadow << 8) | thech;
995           bsLiveShadow += 8;
996           continue;
997         } else {
998           throw new IOException("unexpected end of stream");
999         }
1000       }
1001       bsLiveShadow--;
1002       zvec = (zvec << 1) | ((bsBuffShadow >> bsLiveShadow) & 1);
1003     }
1004 
1005     this.bsLive = bsLiveShadow;
1006     this.bsBuff = bsBuffShadow;
1007 
1008     return dataShadow.perm[zt][zvec - dataShadow.base[zt][zn]];
1009   }
1010 
setupBlock()1011   private void setupBlock() throws IOException {
1012     if (this.data == null) {
1013       return;
1014     }
1015 
1016     final int[] cftab = this.data.cftab;
1017     final int[] tt = this.data.initTT(this.last + 1);
1018     final byte[] ll8 = this.data.ll8;
1019     cftab[0] = 0;
1020     System.arraycopy(this.data.unzftab, 0, cftab, 1, 256);
1021 
1022     for (int i = 1, c = cftab[0]; i <= 256; i++) {
1023       c += cftab[i];
1024       cftab[i] = c;
1025     }
1026 
1027     for (int i = 0, lastShadow = this.last; i <= lastShadow; i++) {
1028       tt[cftab[ll8[i] & 0xff]++] = i;
1029     }
1030 
1031     if ((this.origPtr < 0) || (this.origPtr >= tt.length)) {
1032       throw new IOException("stream corrupted");
1033     }
1034 
1035     this.su_tPos = tt[this.origPtr];
1036     this.su_count = 0;
1037     this.su_i2 = 0;
1038     this.su_ch2 = 256; /* not a char and not EOF */
1039 
1040     if (this.blockRandomised) {
1041       this.su_rNToGo = 0;
1042       this.su_rTPos = 0;
1043       setupRandPartA();
1044     } else {
1045       setupNoRandPartA();
1046     }
1047   }
1048 
setupRandPartA()1049   private void setupRandPartA() throws IOException {
1050     if (this.su_i2 <= this.last) {
1051       this.su_chPrev = this.su_ch2;
1052       int su_ch2Shadow = this.data.ll8[this.su_tPos] & 0xff;
1053       this.su_tPos = this.data.tt[this.su_tPos];
1054       if (this.su_rNToGo == 0) {
1055         this.su_rNToGo = BZip2Constants.rNums[this.su_rTPos] - 1;
1056         if (++this.su_rTPos == 512) {
1057           this.su_rTPos = 0;
1058         }
1059       } else {
1060         this.su_rNToGo--;
1061       }
1062       this.su_ch2 = su_ch2Shadow ^= (this.su_rNToGo == 1) ? 1 : 0;
1063       this.su_i2++;
1064       this.currentChar = su_ch2Shadow;
1065       this.currentState = STATE.RAND_PART_B_STATE;
1066       this.crc.updateCRC(su_ch2Shadow);
1067     } else {
1068       endBlock();
1069       if (readMode == READ_MODE.CONTINUOUS) {
1070       initBlock();
1071       setupBlock();
1072       } else if (readMode == READ_MODE.BYBLOCK) {
1073         this.currentState = STATE.NO_PROCESS_STATE;
1074       }
1075     }
1076   }
1077 
setupNoRandPartA()1078   private void setupNoRandPartA() throws IOException {
1079     if (this.su_i2 <= this.last) {
1080       this.su_chPrev = this.su_ch2;
1081       int su_ch2Shadow = this.data.ll8[this.su_tPos] & 0xff;
1082       this.su_ch2 = su_ch2Shadow;
1083       this.su_tPos = this.data.tt[this.su_tPos];
1084       this.su_i2++;
1085       this.currentChar = su_ch2Shadow;
1086       this.currentState = STATE.NO_RAND_PART_B_STATE;
1087       this.crc.updateCRC(su_ch2Shadow);
1088     } else {
1089       this.currentState = STATE.NO_RAND_PART_A_STATE;
1090       endBlock();
1091       if (readMode == READ_MODE.CONTINUOUS) {
1092       initBlock();
1093       setupBlock();
1094       } else if (readMode == READ_MODE.BYBLOCK) {
1095         this.currentState = STATE.NO_PROCESS_STATE;
1096       }
1097     }
1098   }
1099 
setupRandPartB()1100   private void setupRandPartB() throws IOException {
1101     if (this.su_ch2 != this.su_chPrev) {
1102       this.currentState = STATE.RAND_PART_A_STATE;
1103       this.su_count = 1;
1104       setupRandPartA();
1105     } else if (++this.su_count >= 4) {
1106       this.su_z = (char) (this.data.ll8[this.su_tPos] & 0xff);
1107       this.su_tPos = this.data.tt[this.su_tPos];
1108       if (this.su_rNToGo == 0) {
1109         this.su_rNToGo = BZip2Constants.rNums[this.su_rTPos] - 1;
1110         if (++this.su_rTPos == 512) {
1111           this.su_rTPos = 0;
1112         }
1113       } else {
1114         this.su_rNToGo--;
1115       }
1116       this.su_j2 = 0;
1117       this.currentState = STATE.RAND_PART_C_STATE;
1118       if (this.su_rNToGo == 1) {
1119         this.su_z ^= 1;
1120       }
1121       setupRandPartC();
1122     } else {
1123       this.currentState = STATE.RAND_PART_A_STATE;
1124       setupRandPartA();
1125     }
1126   }
1127 
setupRandPartC()1128   private void setupRandPartC() throws IOException {
1129     if (this.su_j2 < this.su_z) {
1130       this.currentChar = this.su_ch2;
1131       this.crc.updateCRC(this.su_ch2);
1132       this.su_j2++;
1133     } else {
1134       this.currentState = STATE.RAND_PART_A_STATE;
1135       this.su_i2++;
1136       this.su_count = 0;
1137       setupRandPartA();
1138     }
1139   }
1140 
setupNoRandPartB()1141   private void setupNoRandPartB() throws IOException {
1142     if (this.su_ch2 != this.su_chPrev) {
1143       this.su_count = 1;
1144       setupNoRandPartA();
1145     } else if (++this.su_count >= 4) {
1146       this.su_z = (char) (this.data.ll8[this.su_tPos] & 0xff);
1147       this.su_tPos = this.data.tt[this.su_tPos];
1148       this.su_j2 = 0;
1149       setupNoRandPartC();
1150     } else {
1151       setupNoRandPartA();
1152     }
1153   }
1154 
setupNoRandPartC()1155   private void setupNoRandPartC() throws IOException {
1156     if (this.su_j2 < this.su_z) {
1157       int su_ch2Shadow = this.su_ch2;
1158       this.currentChar = su_ch2Shadow;
1159       this.crc.updateCRC(su_ch2Shadow);
1160       this.su_j2++;
1161       this.currentState = STATE.NO_RAND_PART_C_STATE;
1162     } else {
1163       this.su_i2++;
1164       this.su_count = 0;
1165       setupNoRandPartA();
1166     }
1167   }
1168 
1169   private static final class Data extends Object {
1170 
1171     // (with blockSize 900k)
1172     final boolean[] inUse = new boolean[256]; // 256 byte
1173 
1174     final byte[] seqToUnseq = new byte[256]; // 256 byte
1175     final byte[] selector = new byte[MAX_SELECTORS]; // 18002 byte
1176     final byte[] selectorMtf = new byte[MAX_SELECTORS]; // 18002 byte
1177 
1178     /**
1179     * Freq table collected to save a pass over the data during
1180     * decompression.
1181     */
1182     final int[] unzftab = new int[256]; // 1024 byte
1183 
1184     final int[][] limit = new int[N_GROUPS][MAX_ALPHA_SIZE]; // 6192 byte
1185     final int[][] base = new int[N_GROUPS][MAX_ALPHA_SIZE]; // 6192 byte
1186     final int[][] perm = new int[N_GROUPS][MAX_ALPHA_SIZE]; // 6192 byte
1187     final int[] minLens = new int[N_GROUPS]; // 24 byte
1188 
1189     final int[] cftab = new int[257]; // 1028 byte
1190     final char[] getAndMoveToFrontDecode_yy = new char[256]; // 512 byte
1191     final char[][] temp_charArray2d = new char[N_GROUPS][MAX_ALPHA_SIZE]; // 3096
1192                                         // byte
1193     final byte[] recvDecodingTables_pos = new byte[N_GROUPS]; // 6 byte
1194     // ---------------
1195     // 60798 byte
1196 
1197     int[] tt; // 3600000 byte
1198     byte[] ll8; // 900000 byte
1199 
1200     // ---------------
1201     // 4560782 byte
1202     // ===============
1203 
Data(int blockSize100k)1204     Data(int blockSize100k) {
1205       super();
1206 
1207       this.ll8 = new byte[blockSize100k * BZip2Constants.baseBlockSize];
1208     }
1209 
1210     /**
1211     * Initializes the {@link #tt} array.
1212     *
1213     * This method is called when the required length of the array is known.
1214     * I don't initialize it at construction time to avoid unneccessary
1215     * memory allocation when compressing small files.
1216     */
initTT(int length)1217     final int[] initTT(int length) {
1218       int[] ttShadow = this.tt;
1219 
1220       // tt.length should always be >= length, but theoretically
1221       // it can happen, if the compressor mixed small and large
1222       // blocks. Normally only the last block will be smaller
1223       // than others.
1224       if ((ttShadow == null) || (ttShadow.length < length)) {
1225         this.tt = ttShadow = new int[length];
1226       }
1227 
1228       return ttShadow;
1229     }
1230 
1231   }
1232 }
1233