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