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 
25 package org.apache.hadoop.io.compress.bzip2;
26 
27 import java.io.OutputStream;
28 import java.io.IOException;
29 
30 import org.apache.hadoop.io.IOUtils;
31 
32 /**
33  * An output stream that compresses into the BZip2 format (without the file
34  * header chars) into another stream.
35  *
36  * <p>
37  * The compression requires large amounts of memory. Thus you should call the
38  * {@link #close() close()} method as soon as possible, to force
39  * <tt>CBZip2OutputStream</tt> to release the allocated memory.
40  * </p>
41  *
42  * <p>
43  * You can shrink the amount of allocated memory and maybe raise the compression
44  * speed by choosing a lower blocksize, which in turn may cause a lower
45  * compression ratio. You can avoid unnecessary memory allocation by avoiding
46  * using a blocksize which is bigger than the size of the input.
47  * </p>
48  *
49  * <p>
50  * You can compute the memory usage for compressing by the following formula:
51  * </p>
52  *
53  * <pre>
54  * &lt;code&gt;400k + (9 * blocksize)&lt;/code&gt;.
55  * </pre>
56  *
57  * <p>
58  * To get the memory required for decompression by {@link CBZip2InputStream
59  * CBZip2InputStream} use
60  * </p>
61  *
62  * <pre>
63  * &lt;code&gt;65k + (5 * blocksize)&lt;/code&gt;.
64  * </pre>
65  *
66  * <table width="100%" border="1">
67  * <colgroup> <col width="33%" /> <col width="33%" /> <col width="33%" />
68  * </colgroup>
69  * <tr>
70  * <th colspan="3">Memory usage by blocksize</th>
71  * </tr>
72  * <tr>
73  * <th align="right">Blocksize</th> <th align="right">Compression<br>
74  * memory usage</th> <th align="right">Decompression<br>
75  * memory usage</th>
76  * </tr>
77  * <tr>
78  * <td align="right">100k</td>
79  * <td align="right">1300k</td>
80  * <td align="right">565k</td>
81  * </tr>
82  * <tr>
83  * <td align="right">200k</td>
84  * <td align="right">2200k</td>
85  * <td align="right">1065k</td>
86  * </tr>
87  * <tr>
88  * <td align="right">300k</td>
89  * <td align="right">3100k</td>
90  * <td align="right">1565k</td>
91  * </tr>
92  * <tr>
93  * <td align="right">400k</td>
94  * <td align="right">4000k</td>
95  * <td align="right">2065k</td>
96  * </tr>
97  * <tr>
98  * <td align="right">500k</td>
99  * <td align="right">4900k</td>
100  * <td align="right">2565k</td>
101  * </tr>
102  * <tr>
103  * <td align="right">600k</td>
104  * <td align="right">5800k</td>
105  * <td align="right">3065k</td>
106  * </tr>
107  * <tr>
108  * <td align="right">700k</td>
109  * <td align="right">6700k</td>
110  * <td align="right">3565k</td>
111  * </tr>
112  * <tr>
113  * <td align="right">800k</td>
114  * <td align="right">7600k</td>
115  * <td align="right">4065k</td>
116  * </tr>
117  * <tr>
118  * <td align="right">900k</td>
119  * <td align="right">8500k</td>
120  * <td align="right">4565k</td>
121  * </tr>
122  * </table>
123  *
124  * <p>
125  * For decompression <tt>CBZip2InputStream</tt> allocates less memory if the
126  * bzipped input is smaller than one block.
127  * </p>
128  *
129  * <p>
130  * Instances of this class are not threadsafe.
131  * </p>
132  *
133  * <p>
134  * TODO: Update to BZip2 1.0.1
135  * </p>
136  *
137  */
138 public class CBZip2OutputStream extends OutputStream implements BZip2Constants {
139 
140   /**
141   * The minimum supported blocksize <tt> == 1</tt>.
142   */
143   public static final int MIN_BLOCKSIZE = 1;
144 
145   /**
146   * The maximum supported blocksize <tt> == 9</tt>.
147   */
148   public static final int MAX_BLOCKSIZE = 9;
149 
150   /**
151   * This constant is accessible by subclasses for historical purposes. If you
152   * don't know what it means then you don't need it.
153   */
154   protected static final int SETMASK = (1 << 21);
155 
156   /**
157   * This constant is accessible by subclasses for historical purposes. If you
158   * don't know what it means then you don't need it.
159   */
160   protected static final int CLEARMASK = (~SETMASK);
161 
162   /**
163   * This constant is accessible by subclasses for historical purposes. If you
164   * don't know what it means then you don't need it.
165   */
166   protected static final int GREATER_ICOST = 15;
167 
168   /**
169   * This constant is accessible by subclasses for historical purposes. If you
170   * don't know what it means then you don't need it.
171   */
172   protected static final int LESSER_ICOST = 0;
173 
174   /**
175   * This constant is accessible by subclasses for historical purposes. If you
176   * don't know what it means then you don't need it.
177   */
178   protected static final int SMALL_THRESH = 20;
179 
180   /**
181   * This constant is accessible by subclasses for historical purposes. If you
182   * don't know what it means then you don't need it.
183   */
184   protected static final int DEPTH_THRESH = 10;
185 
186   /**
187   * This constant is accessible by subclasses for historical purposes. If you
188   * don't know what it means then you don't need it.
189   */
190   protected static final int WORK_FACTOR = 30;
191 
192   /**
193   * This constant is accessible by subclasses for historical purposes. If you
194   * don't know what it means then you don't need it.
195   * <p>
196   * If you are ever unlucky/improbable enough to get a stack overflow whilst
197   * sorting, increase the following constant and try again. In practice I
198   * have never seen the stack go above 27 elems, so the following limit seems
199   * very generous.
200   * </p>
201   */
202   protected static final int QSORT_STACK_SIZE = 1000;
203 
204   /**
205   * Knuth's increments seem to work better than Incerpi-Sedgewick here.
206   * Possibly because the number of elems to sort is usually small, typically
207   * &lt;= 20.
208   */
209   private static final int[] INCS = { 1, 4, 13, 40, 121, 364, 1093, 3280,
210       9841, 29524, 88573, 265720, 797161, 2391484 };
211 
212   /**
213   * This method is accessible by subclasses for historical purposes. If you
214   * don't know what it does then you don't need it.
215   */
hbMakeCodeLengths(char[] len, int[] freq, int alphaSize, int maxLen)216   protected static void hbMakeCodeLengths(char[] len, int[] freq,
217       int alphaSize, int maxLen) {
218     /*
219     * Nodes and heap entries run from 1. Entry 0 for both the heap and
220     * nodes is a sentinel.
221     */
222     final int[] heap = new int[MAX_ALPHA_SIZE * 2];
223     final int[] weight = new int[MAX_ALPHA_SIZE * 2];
224     final int[] parent = new int[MAX_ALPHA_SIZE * 2];
225 
226     for (int i = alphaSize; --i >= 0;) {
227       weight[i + 1] = (freq[i] == 0 ? 1 : freq[i]) << 8;
228     }
229 
230     for (boolean tooLong = true; tooLong;) {
231       tooLong = false;
232 
233       int nNodes = alphaSize;
234       int nHeap = 0;
235       heap[0] = 0;
236       weight[0] = 0;
237       parent[0] = -2;
238 
239       for (int i = 1; i <= alphaSize; i++) {
240         parent[i] = -1;
241         nHeap++;
242         heap[nHeap] = i;
243 
244         int zz = nHeap;
245         int tmp = heap[zz];
246         while (weight[tmp] < weight[heap[zz >> 1]]) {
247           heap[zz] = heap[zz >> 1];
248           zz >>= 1;
249         }
250         heap[zz] = tmp;
251       }
252 
253       // assert (nHeap < (MAX_ALPHA_SIZE + 2)) : nHeap;
254 
255       while (nHeap > 1) {
256         int n1 = heap[1];
257         heap[1] = heap[nHeap];
258         nHeap--;
259 
260         int yy = 0;
261         int zz = 1;
262         int tmp = heap[1];
263 
264         while (true) {
265           yy = zz << 1;
266 
267           if (yy > nHeap) {
268             break;
269           }
270 
271           if ((yy < nHeap)
272               && (weight[heap[yy + 1]] < weight[heap[yy]])) {
273             yy++;
274           }
275 
276           if (weight[tmp] < weight[heap[yy]]) {
277             break;
278           }
279 
280           heap[zz] = heap[yy];
281           zz = yy;
282         }
283 
284         heap[zz] = tmp;
285 
286         int n2 = heap[1];
287         heap[1] = heap[nHeap];
288         nHeap--;
289 
290         yy = 0;
291         zz = 1;
292         tmp = heap[1];
293 
294         while (true) {
295           yy = zz << 1;
296 
297           if (yy > nHeap) {
298             break;
299           }
300 
301           if ((yy < nHeap)
302               && (weight[heap[yy + 1]] < weight[heap[yy]])) {
303             yy++;
304           }
305 
306           if (weight[tmp] < weight[heap[yy]]) {
307             break;
308           }
309 
310           heap[zz] = heap[yy];
311           zz = yy;
312         }
313 
314         heap[zz] = tmp;
315         nNodes++;
316         parent[n1] = parent[n2] = nNodes;
317 
318         final int weight_n1 = weight[n1];
319         final int weight_n2 = weight[n2];
320         weight[nNodes] = (((weight_n1 & 0xffffff00) + (weight_n2 & 0xffffff00)) | (1 + (((weight_n1 & 0x000000ff) > (weight_n2 & 0x000000ff)) ? (weight_n1 & 0x000000ff)
321             : (weight_n2 & 0x000000ff))));
322 
323         parent[nNodes] = -1;
324         nHeap++;
325         heap[nHeap] = nNodes;
326 
327         tmp = 0;
328         zz = nHeap;
329         tmp = heap[zz];
330         final int weight_tmp = weight[tmp];
331         while (weight_tmp < weight[heap[zz >> 1]]) {
332           heap[zz] = heap[zz >> 1];
333           zz >>= 1;
334         }
335         heap[zz] = tmp;
336 
337       }
338 
339       // assert (nNodes < (MAX_ALPHA_SIZE * 2)) : nNodes;
340 
341       for (int i = 1; i <= alphaSize; i++) {
342         int j = 0;
343         int k = i;
344 
345         for (int parent_k; (parent_k = parent[k]) >= 0;) {
346           k = parent_k;
347           j++;
348         }
349 
350         len[i - 1] = (char) j;
351         if (j > maxLen) {
352           tooLong = true;
353         }
354       }
355 
356       if (tooLong) {
357         for (int i = 1; i < alphaSize; i++) {
358           int j = weight[i] >> 8;
359           j = 1 + (j >> 1);
360           weight[i] = j << 8;
361         }
362       }
363     }
364   }
365 
hbMakeCodeLengths(final byte[] len, final int[] freq, final Data dat, final int alphaSize, final int maxLen)366   private static void hbMakeCodeLengths(final byte[] len, final int[] freq,
367       final Data dat, final int alphaSize, final int maxLen) {
368     /*
369     * Nodes and heap entries run from 1. Entry 0 for both the heap and
370     * nodes is a sentinel.
371     */
372     final int[] heap = dat.heap;
373     final int[] weight = dat.weight;
374     final int[] parent = dat.parent;
375 
376     for (int i = alphaSize; --i >= 0;) {
377       weight[i + 1] = (freq[i] == 0 ? 1 : freq[i]) << 8;
378     }
379 
380     for (boolean tooLong = true; tooLong;) {
381       tooLong = false;
382 
383       int nNodes = alphaSize;
384       int nHeap = 0;
385       heap[0] = 0;
386       weight[0] = 0;
387       parent[0] = -2;
388 
389       for (int i = 1; i <= alphaSize; i++) {
390         parent[i] = -1;
391         nHeap++;
392         heap[nHeap] = i;
393 
394         int zz = nHeap;
395         int tmp = heap[zz];
396         while (weight[tmp] < weight[heap[zz >> 1]]) {
397           heap[zz] = heap[zz >> 1];
398           zz >>= 1;
399         }
400         heap[zz] = tmp;
401       }
402 
403       while (nHeap > 1) {
404         int n1 = heap[1];
405         heap[1] = heap[nHeap];
406         nHeap--;
407 
408         int yy = 0;
409         int zz = 1;
410         int tmp = heap[1];
411 
412         while (true) {
413           yy = zz << 1;
414 
415           if (yy > nHeap) {
416             break;
417           }
418 
419           if ((yy < nHeap)
420               && (weight[heap[yy + 1]] < weight[heap[yy]])) {
421             yy++;
422           }
423 
424           if (weight[tmp] < weight[heap[yy]]) {
425             break;
426           }
427 
428           heap[zz] = heap[yy];
429           zz = yy;
430         }
431 
432         heap[zz] = tmp;
433 
434         int n2 = heap[1];
435         heap[1] = heap[nHeap];
436         nHeap--;
437 
438         yy = 0;
439         zz = 1;
440         tmp = heap[1];
441 
442         while (true) {
443           yy = zz << 1;
444 
445           if (yy > nHeap) {
446             break;
447           }
448 
449           if ((yy < nHeap)
450               && (weight[heap[yy + 1]] < weight[heap[yy]])) {
451             yy++;
452           }
453 
454           if (weight[tmp] < weight[heap[yy]]) {
455             break;
456           }
457 
458           heap[zz] = heap[yy];
459           zz = yy;
460         }
461 
462         heap[zz] = tmp;
463         nNodes++;
464         parent[n1] = parent[n2] = nNodes;
465 
466         final int weight_n1 = weight[n1];
467         final int weight_n2 = weight[n2];
468         weight[nNodes] = ((weight_n1 & 0xffffff00) + (weight_n2 & 0xffffff00))
469             | (1 + (((weight_n1 & 0x000000ff) > (weight_n2 & 0x000000ff)) ? (weight_n1 & 0x000000ff)
470                 : (weight_n2 & 0x000000ff)));
471 
472         parent[nNodes] = -1;
473         nHeap++;
474         heap[nHeap] = nNodes;
475 
476         tmp = 0;
477         zz = nHeap;
478         tmp = heap[zz];
479         final int weight_tmp = weight[tmp];
480         while (weight_tmp < weight[heap[zz >> 1]]) {
481           heap[zz] = heap[zz >> 1];
482           zz >>= 1;
483         }
484         heap[zz] = tmp;
485 
486       }
487 
488       for (int i = 1; i <= alphaSize; i++) {
489         int j = 0;
490         int k = i;
491 
492         for (int parent_k; (parent_k = parent[k]) >= 0;) {
493           k = parent_k;
494           j++;
495         }
496 
497         len[i - 1] = (byte) j;
498         if (j > maxLen) {
499           tooLong = true;
500         }
501       }
502 
503       if (tooLong) {
504         for (int i = 1; i < alphaSize; i++) {
505           int j = weight[i] >> 8;
506           j = 1 + (j >> 1);
507           weight[i] = j << 8;
508         }
509       }
510     }
511   }
512 
513   /**
514   * Index of the last char in the block, so the block size == last + 1.
515   */
516   private int last;
517 
518   /**
519   * Index in fmap[] of original string after sorting.
520   */
521   private int origPtr;
522 
523   /**
524   * Always: in the range 0 .. 9. The current block size is 100000 * this
525   * number.
526   */
527   private final int blockSize100k;
528 
529   private boolean blockRandomised;
530 
531   private int bsBuff;
532   private int bsLive;
533   private final CRC crc = new CRC();
534 
535   private int nInUse;
536 
537   private int nMTF;
538 
539   /*
540   * Used when sorting. If too many long comparisons happen, we stop sorting,
541   * randomise the block slightly, and try again.
542   */
543   private int workDone;
544   private int workLimit;
545   private boolean firstAttempt;
546 
547   private int currentChar = -1;
548   private int runLength = 0;
549 
550   private int blockCRC;
551   private int combinedCRC;
552   private int allowableBlockSize;
553 
554   /**
555   * All memory intensive stuff.
556   */
557   private CBZip2OutputStream.Data data;
558 
559   private OutputStream out;
560 
561   /**
562   * Chooses a blocksize based on the given length of the data to compress.
563   *
564   * @return The blocksize, between {@link #MIN_BLOCKSIZE} and
565   *         {@link #MAX_BLOCKSIZE} both inclusive. For a negative
566   *         <tt>inputLength</tt> this method returns <tt>MAX_BLOCKSIZE</tt>
567   *         always.
568   *
569   * @param inputLength
570   *            The length of the data which will be compressed by
571   *            <tt>CBZip2OutputStream</tt>.
572   */
chooseBlockSize(long inputLength)573   public static int chooseBlockSize(long inputLength) {
574     return (inputLength > 0) ? (int) Math
575         .min((inputLength / 132000) + 1, 9) : MAX_BLOCKSIZE;
576   }
577 
578   /**
579   * Constructs a new <tt>CBZip2OutputStream</tt> with a blocksize of 900k.
580   *
581   * <p>
582   * <b>Attention: </b>The caller is resonsible to write the two BZip2 magic
583   * bytes <tt>"BZ"</tt> to the specified stream prior to calling this
584   * constructor.
585   * </p>
586   *
587   * @param out *
588   *            the destination stream.
589   *
590   * @throws IOException
591   *             if an I/O error occurs in the specified stream.
592   * @throws NullPointerException
593   *             if <code>out == null</code>.
594   */
CBZip2OutputStream(final OutputStream out)595   public CBZip2OutputStream(final OutputStream out) throws IOException {
596     this(out, MAX_BLOCKSIZE);
597   }
598 
599   /**
600   * Constructs a new <tt>CBZip2OutputStream</tt> with specified blocksize.
601   *
602   * <p>
603   * <b>Attention: </b>The caller is resonsible to write the two BZip2 magic
604   * bytes <tt>"BZ"</tt> to the specified stream prior to calling this
605   * constructor.
606   * </p>
607   *
608   *
609   * @param out
610   *            the destination stream.
611   * @param blockSize
612   *            the blockSize as 100k units.
613   *
614   * @throws IOException
615   *             if an I/O error occurs in the specified stream.
616   * @throws IllegalArgumentException
617   *             if <code>(blockSize < 1) || (blockSize > 9)</code>.
618   * @throws NullPointerException
619   *             if <code>out == null</code>.
620   *
621   * @see #MIN_BLOCKSIZE
622   * @see #MAX_BLOCKSIZE
623   */
CBZip2OutputStream(final OutputStream out, final int blockSize)624   public CBZip2OutputStream(final OutputStream out, final int blockSize)
625       throws IOException {
626     super();
627 
628     if (blockSize < 1) {
629       throw new IllegalArgumentException("blockSize(" + blockSize
630           + ") < 1");
631     }
632     if (blockSize > 9) {
633       throw new IllegalArgumentException("blockSize(" + blockSize
634           + ") > 9");
635     }
636 
637     this.blockSize100k = blockSize;
638     this.out = out;
639     init();
640   }
641 
642   @Override
write(final int b)643   public void write(final int b) throws IOException {
644     if (this.out != null) {
645       write0(b);
646     } else {
647       throw new IOException("closed");
648     }
649   }
650 
writeRun()651   private void writeRun() throws IOException {
652     final int lastShadow = this.last;
653 
654     if (lastShadow < this.allowableBlockSize) {
655       final int currentCharShadow = this.currentChar;
656       final Data dataShadow = this.data;
657       dataShadow.inUse[currentCharShadow] = true;
658       final byte ch = (byte) currentCharShadow;
659 
660       int runLengthShadow = this.runLength;
661       this.crc.updateCRC(currentCharShadow, runLengthShadow);
662 
663       switch (runLengthShadow) {
664       case 1:
665         dataShadow.block[lastShadow + 2] = ch;
666         this.last = lastShadow + 1;
667         break;
668 
669       case 2:
670         dataShadow.block[lastShadow + 2] = ch;
671         dataShadow.block[lastShadow + 3] = ch;
672         this.last = lastShadow + 2;
673         break;
674 
675       case 3: {
676         final byte[] block = dataShadow.block;
677         block[lastShadow + 2] = ch;
678         block[lastShadow + 3] = ch;
679         block[lastShadow + 4] = ch;
680         this.last = lastShadow + 3;
681       }
682         break;
683 
684       default: {
685         runLengthShadow -= 4;
686         dataShadow.inUse[runLengthShadow] = true;
687         final byte[] block = dataShadow.block;
688         block[lastShadow + 2] = ch;
689         block[lastShadow + 3] = ch;
690         block[lastShadow + 4] = ch;
691         block[lastShadow + 5] = ch;
692         block[lastShadow + 6] = (byte) runLengthShadow;
693         this.last = lastShadow + 5;
694       }
695         break;
696 
697       }
698     } else {
699       endBlock();
700       initBlock();
701       writeRun();
702     }
703   }
704 
705   /**
706   * Overriden to close the stream.
707   */
708   @Override
finalize()709   protected void finalize() throws Throwable {
710     finish();
711     super.finalize();
712   }
713 
714 
finish()715   public void finish() throws IOException {
716     if (out != null) {
717       try {
718         if (this.runLength > 0) {
719           writeRun();
720         }
721         this.currentChar = -1;
722         endBlock();
723         endCompression();
724       } finally {
725         this.out = null;
726         this.data = null;
727       }
728     }
729   }
730 
731   @Override
close()732   public void close() throws IOException {
733     if (out != null) {
734       OutputStream outShadow = this.out;
735       try {
736         finish();
737         outShadow.close();
738         outShadow = null;
739       } finally {
740         IOUtils.closeStream(outShadow);
741       }
742     }
743   }
744 
745   @Override
flush()746   public void flush() throws IOException {
747     OutputStream outShadow = this.out;
748     if (outShadow != null) {
749       outShadow.flush();
750     }
751   }
752 
init()753   private void init() throws IOException {
754     // write magic: done by caller who created this stream
755     // this.out.write('B');
756     // this.out.write('Z');
757 
758     this.data = new Data(this.blockSize100k);
759 
760     /*
761     * Write `magic' bytes h indicating file-format == huffmanised, followed
762     * by a digit indicating blockSize100k.
763     */
764     bsPutUByte('h');
765     bsPutUByte('0' + this.blockSize100k);
766 
767     this.combinedCRC = 0;
768     initBlock();
769   }
770 
initBlock()771   private void initBlock() {
772     // blockNo++;
773     this.crc.initialiseCRC();
774     this.last = -1;
775     // ch = 0;
776 
777     boolean[] inUse = this.data.inUse;
778     for (int i = 256; --i >= 0;) {
779       inUse[i] = false;
780     }
781 
782     /* 20 is just a paranoia constant */
783     this.allowableBlockSize = (this.blockSize100k * BZip2Constants.baseBlockSize) - 20;
784   }
785 
endBlock()786   private void endBlock() throws IOException {
787     this.blockCRC = this.crc.getFinalCRC();
788     this.combinedCRC = (this.combinedCRC << 1) | (this.combinedCRC >>> 31);
789     this.combinedCRC ^= this.blockCRC;
790 
791     // empty block at end of file
792     if (this.last == -1) {
793       return;
794     }
795 
796     /* sort the block and establish posn of original string */
797     blockSort();
798 
799     /*
800     * A 6-byte block header, the value chosen arbitrarily as 0x314159265359
801     * :-). A 32 bit value does not really give a strong enough guarantee
802     * that the value will not appear by chance in the compressed
803     * datastream. Worst-case probability of this event, for a 900k block,
804     * is about 2.0e-3 for 32 bits, 1.0e-5 for 40 bits and 4.0e-8 for 48
805     * bits. For a compressed file of size 100Gb -- about 100000 blocks --
806     * only a 48-bit marker will do. NB: normal compression/ decompression
807     * donot rely on these statistical properties. They are only important
808     * when trying to recover blocks from damaged files.
809     */
810     bsPutUByte(0x31);
811     bsPutUByte(0x41);
812     bsPutUByte(0x59);
813     bsPutUByte(0x26);
814     bsPutUByte(0x53);
815     bsPutUByte(0x59);
816 
817     /* Now the block's CRC, so it is in a known place. */
818     bsPutInt(this.blockCRC);
819 
820     /* Now a single bit indicating randomisation. */
821     if (this.blockRandomised) {
822       bsW(1, 1);
823     } else {
824       bsW(1, 0);
825     }
826 
827     /* Finally, block's contents proper. */
828     moveToFrontCodeAndSend();
829   }
830 
endCompression()831   private void endCompression() throws IOException {
832     /*
833     * Now another magic 48-bit number, 0x177245385090, to indicate the end
834     * of the last block. (sqrt(pi), if you want to know. I did want to use
835     * e, but it contains too much repetition -- 27 18 28 18 28 46 -- for me
836     * to feel statistically comfortable. Call me paranoid.)
837     */
838     bsPutUByte(0x17);
839     bsPutUByte(0x72);
840     bsPutUByte(0x45);
841     bsPutUByte(0x38);
842     bsPutUByte(0x50);
843     bsPutUByte(0x90);
844 
845     bsPutInt(this.combinedCRC);
846     bsFinishedWithStream();
847   }
848 
849   /**
850   * Returns the blocksize parameter specified at construction time.
851   */
getBlockSize()852   public final int getBlockSize() {
853     return this.blockSize100k;
854   }
855 
856   @Override
write(final byte[] buf, int offs, final int len)857   public void write(final byte[] buf, int offs, final int len)
858       throws IOException {
859     if (offs < 0) {
860       throw new IndexOutOfBoundsException("offs(" + offs + ") < 0.");
861     }
862     if (len < 0) {
863       throw new IndexOutOfBoundsException("len(" + len + ") < 0.");
864     }
865     if (offs + len > buf.length) {
866       throw new IndexOutOfBoundsException("offs(" + offs + ") + len("
867           + len + ") > buf.length(" + buf.length + ").");
868     }
869     if (this.out == null) {
870       throw new IOException("stream closed");
871     }
872 
873     for (int hi = offs + len; offs < hi;) {
874       write0(buf[offs++]);
875     }
876   }
877 
write0(int b)878   private void write0(int b) throws IOException {
879     if (this.currentChar != -1) {
880       b &= 0xff;
881       if (this.currentChar == b) {
882         if (++this.runLength > 254) {
883           writeRun();
884           this.currentChar = -1;
885           this.runLength = 0;
886         }
887         // else nothing to do
888       } else {
889         writeRun();
890         this.runLength = 1;
891         this.currentChar = b;
892       }
893     } else {
894       this.currentChar = b & 0xff;
895       this.runLength++;
896     }
897   }
898 
hbAssignCodes(final int[] code, final byte[] length, final int minLen, final int maxLen, final int alphaSize)899   private static void hbAssignCodes(final int[] code, final byte[] length,
900       final int minLen, final int maxLen, final int alphaSize) {
901     int vec = 0;
902     for (int n = minLen; n <= maxLen; n++) {
903       for (int i = 0; i < alphaSize; i++) {
904         if ((length[i] & 0xff) == n) {
905           code[i] = vec;
906           vec++;
907         }
908       }
909       vec <<= 1;
910     }
911   }
912 
bsFinishedWithStream()913   private void bsFinishedWithStream() throws IOException {
914     while (this.bsLive > 0) {
915       int ch = this.bsBuff >> 24;
916       this.out.write(ch); // write 8-bit
917       this.bsBuff <<= 8;
918       this.bsLive -= 8;
919     }
920   }
921 
bsW(final int n, final int v)922   private void bsW(final int n, final int v) throws IOException {
923     final OutputStream outShadow = this.out;
924     int bsLiveShadow = this.bsLive;
925     int bsBuffShadow = this.bsBuff;
926 
927     while (bsLiveShadow >= 8) {
928       outShadow.write(bsBuffShadow >> 24); // write 8-bit
929       bsBuffShadow <<= 8;
930       bsLiveShadow -= 8;
931     }
932 
933     this.bsBuff = bsBuffShadow | (v << (32 - bsLiveShadow - n));
934     this.bsLive = bsLiveShadow + n;
935   }
936 
bsPutUByte(final int c)937   private void bsPutUByte(final int c) throws IOException {
938     bsW(8, c);
939   }
940 
bsPutInt(final int u)941   private void bsPutInt(final int u) throws IOException {
942     bsW(8, (u >> 24) & 0xff);
943     bsW(8, (u >> 16) & 0xff);
944     bsW(8, (u >> 8) & 0xff);
945     bsW(8, u & 0xff);
946   }
947 
sendMTFValues()948   private void sendMTFValues() throws IOException {
949     final byte[][] len = this.data.sendMTFValues_len;
950     final int alphaSize = this.nInUse + 2;
951 
952     for (int t = N_GROUPS; --t >= 0;) {
953       byte[] len_t = len[t];
954       for (int v = alphaSize; --v >= 0;) {
955         len_t[v] = GREATER_ICOST;
956       }
957     }
958 
959     /* Decide how many coding tables to use */
960     // assert (this.nMTF > 0) : this.nMTF;
961     final int nGroups = (this.nMTF < 200) ? 2 : (this.nMTF < 600) ? 3
962         : (this.nMTF < 1200) ? 4 : (this.nMTF < 2400) ? 5 : 6;
963 
964     /* Generate an initial set of coding tables */
965     sendMTFValues0(nGroups, alphaSize);
966 
967     /*
968     * Iterate up to N_ITERS times to improve the tables.
969     */
970     final int nSelectors = sendMTFValues1(nGroups, alphaSize);
971 
972     /* Compute MTF values for the selectors. */
973     sendMTFValues2(nGroups, nSelectors);
974 
975     /* Assign actual codes for the tables. */
976     sendMTFValues3(nGroups, alphaSize);
977 
978     /* Transmit the mapping table. */
979     sendMTFValues4();
980 
981     /* Now the selectors. */
982     sendMTFValues5(nGroups, nSelectors);
983 
984     /* Now the coding tables. */
985     sendMTFValues6(nGroups, alphaSize);
986 
987     /* And finally, the block data proper */
988     sendMTFValues7(nSelectors);
989   }
990 
sendMTFValues0(final int nGroups, final int alphaSize)991   private void sendMTFValues0(final int nGroups, final int alphaSize) {
992     final byte[][] len = this.data.sendMTFValues_len;
993     final int[] mtfFreq = this.data.mtfFreq;
994 
995     int remF = this.nMTF;
996     int gs = 0;
997 
998     for (int nPart = nGroups; nPart > 0; nPart--) {
999       final int tFreq = remF / nPart;
1000       int ge = gs - 1;
1001       int aFreq = 0;
1002 
1003       for (final int a = alphaSize - 1; (aFreq < tFreq) && (ge < a);) {
1004         aFreq += mtfFreq[++ge];
1005       }
1006 
1007       if ((ge > gs) && (nPart != nGroups) && (nPart != 1)
1008           && (((nGroups - nPart) & 1) != 0)) {
1009         aFreq -= mtfFreq[ge--];
1010       }
1011 
1012       final byte[] len_np = len[nPart - 1];
1013       for (int v = alphaSize; --v >= 0;) {
1014         if ((v >= gs) && (v <= ge)) {
1015           len_np[v] = LESSER_ICOST;
1016         } else {
1017           len_np[v] = GREATER_ICOST;
1018         }
1019       }
1020 
1021       gs = ge + 1;
1022       remF -= aFreq;
1023     }
1024   }
1025 
sendMTFValues1(final int nGroups, final int alphaSize)1026   private int sendMTFValues1(final int nGroups, final int alphaSize) {
1027     final Data dataShadow = this.data;
1028     final int[][] rfreq = dataShadow.sendMTFValues_rfreq;
1029     final int[] fave = dataShadow.sendMTFValues_fave;
1030     final short[] cost = dataShadow.sendMTFValues_cost;
1031     final char[] sfmap = dataShadow.sfmap;
1032     final byte[] selector = dataShadow.selector;
1033     final byte[][] len = dataShadow.sendMTFValues_len;
1034     final byte[] len_0 = len[0];
1035     final byte[] len_1 = len[1];
1036     final byte[] len_2 = len[2];
1037     final byte[] len_3 = len[3];
1038     final byte[] len_4 = len[4];
1039     final byte[] len_5 = len[5];
1040     final int nMTFShadow = this.nMTF;
1041 
1042     int nSelectors = 0;
1043 
1044     for (int iter = 0; iter < N_ITERS; iter++) {
1045       for (int t = nGroups; --t >= 0;) {
1046         fave[t] = 0;
1047         int[] rfreqt = rfreq[t];
1048         for (int i = alphaSize; --i >= 0;) {
1049           rfreqt[i] = 0;
1050         }
1051       }
1052 
1053       nSelectors = 0;
1054 
1055       for (int gs = 0; gs < this.nMTF;) {
1056         /* Set group start & end marks. */
1057 
1058         /*
1059         * Calculate the cost of this group as coded by each of the
1060         * coding tables.
1061         */
1062 
1063         final int ge = Math.min(gs + G_SIZE - 1, nMTFShadow - 1);
1064 
1065         if (nGroups == N_GROUPS) {
1066           // unrolled version of the else-block
1067 
1068           short cost0 = 0;
1069           short cost1 = 0;
1070           short cost2 = 0;
1071           short cost3 = 0;
1072           short cost4 = 0;
1073           short cost5 = 0;
1074 
1075           for (int i = gs; i <= ge; i++) {
1076             final int icv = sfmap[i];
1077             cost0 += len_0[icv] & 0xff;
1078             cost1 += len_1[icv] & 0xff;
1079             cost2 += len_2[icv] & 0xff;
1080             cost3 += len_3[icv] & 0xff;
1081             cost4 += len_4[icv] & 0xff;
1082             cost5 += len_5[icv] & 0xff;
1083           }
1084 
1085           cost[0] = cost0;
1086           cost[1] = cost1;
1087           cost[2] = cost2;
1088           cost[3] = cost3;
1089           cost[4] = cost4;
1090           cost[5] = cost5;
1091 
1092         } else {
1093           for (int t = nGroups; --t >= 0;) {
1094             cost[t] = 0;
1095           }
1096 
1097           for (int i = gs; i <= ge; i++) {
1098             final int icv = sfmap[i];
1099             for (int t = nGroups; --t >= 0;) {
1100               cost[t] += len[t][icv] & 0xff;
1101             }
1102           }
1103         }
1104 
1105         /*
1106         * Find the coding table which is best for this group, and
1107         * record its identity in the selector table.
1108         */
1109         int bt = -1;
1110         for (int t = nGroups, bc = 999999999; --t >= 0;) {
1111           final int cost_t = cost[t];
1112           if (cost_t < bc) {
1113             bc = cost_t;
1114             bt = t;
1115           }
1116         }
1117 
1118         fave[bt]++;
1119         selector[nSelectors] = (byte) bt;
1120         nSelectors++;
1121 
1122         /*
1123         * Increment the symbol frequencies for the selected table.
1124         */
1125         final int[] rfreq_bt = rfreq[bt];
1126         for (int i = gs; i <= ge; i++) {
1127           rfreq_bt[sfmap[i]]++;
1128         }
1129 
1130         gs = ge + 1;
1131       }
1132 
1133       /*
1134       * Recompute the tables based on the accumulated frequencies.
1135       */
1136       for (int t = 0; t < nGroups; t++) {
1137         hbMakeCodeLengths(len[t], rfreq[t], this.data, alphaSize, 20);
1138       }
1139     }
1140 
1141     return nSelectors;
1142   }
1143 
sendMTFValues2(final int nGroups, final int nSelectors)1144   private void sendMTFValues2(final int nGroups, final int nSelectors) {
1145     // assert (nGroups < 8) : nGroups;
1146 
1147     final Data dataShadow = this.data;
1148     byte[] pos = dataShadow.sendMTFValues2_pos;
1149 
1150     for (int i = nGroups; --i >= 0;) {
1151       pos[i] = (byte) i;
1152     }
1153 
1154     for (int i = 0; i < nSelectors; i++) {
1155       final byte ll_i = dataShadow.selector[i];
1156       byte tmp = pos[0];
1157       int j = 0;
1158 
1159       while (ll_i != tmp) {
1160         j++;
1161         byte tmp2 = tmp;
1162         tmp = pos[j];
1163         pos[j] = tmp2;
1164       }
1165 
1166       pos[0] = tmp;
1167       dataShadow.selectorMtf[i] = (byte) j;
1168     }
1169   }
1170 
sendMTFValues3(final int nGroups, final int alphaSize)1171   private void sendMTFValues3(final int nGroups, final int alphaSize) {
1172     int[][] code = this.data.sendMTFValues_code;
1173     byte[][] len = this.data.sendMTFValues_len;
1174 
1175     for (int t = 0; t < nGroups; t++) {
1176       int minLen = 32;
1177       int maxLen = 0;
1178       final byte[] len_t = len[t];
1179       for (int i = alphaSize; --i >= 0;) {
1180         final int l = len_t[i] & 0xff;
1181         if (l > maxLen) {
1182           maxLen = l;
1183         }
1184         if (l < minLen) {
1185           minLen = l;
1186         }
1187       }
1188 
1189       // assert (maxLen <= 20) : maxLen;
1190       // assert (minLen >= 1) : minLen;
1191 
1192       hbAssignCodes(code[t], len[t], minLen, maxLen, alphaSize);
1193     }
1194   }
1195 
sendMTFValues4()1196   private void sendMTFValues4() throws IOException {
1197     final boolean[] inUse = this.data.inUse;
1198     final boolean[] inUse16 = this.data.sentMTFValues4_inUse16;
1199 
1200     for (int i = 16; --i >= 0;) {
1201       inUse16[i] = false;
1202       final int i16 = i * 16;
1203       for (int j = 16; --j >= 0;) {
1204         if (inUse[i16 + j]) {
1205           inUse16[i] = true;
1206         }
1207       }
1208     }
1209 
1210     for (int i = 0; i < 16; i++) {
1211       bsW(1, inUse16[i] ? 1 : 0);
1212     }
1213 
1214     final OutputStream outShadow = this.out;
1215     int bsLiveShadow = this.bsLive;
1216     int bsBuffShadow = this.bsBuff;
1217 
1218     for (int i = 0; i < 16; i++) {
1219       if (inUse16[i]) {
1220         final int i16 = i * 16;
1221         for (int j = 0; j < 16; j++) {
1222           // inlined: bsW(1, inUse[i16 + j] ? 1 : 0);
1223           while (bsLiveShadow >= 8) {
1224             outShadow.write(bsBuffShadow >> 24); // write 8-bit
1225             bsBuffShadow <<= 8;
1226             bsLiveShadow -= 8;
1227           }
1228           if (inUse[i16 + j]) {
1229             bsBuffShadow |= 1 << (32 - bsLiveShadow - 1);
1230           }
1231           bsLiveShadow++;
1232         }
1233       }
1234     }
1235 
1236     this.bsBuff = bsBuffShadow;
1237     this.bsLive = bsLiveShadow;
1238   }
1239 
sendMTFValues5(final int nGroups, final int nSelectors)1240   private void sendMTFValues5(final int nGroups, final int nSelectors)
1241       throws IOException {
1242     bsW(3, nGroups);
1243     bsW(15, nSelectors);
1244 
1245     final OutputStream outShadow = this.out;
1246     final byte[] selectorMtf = this.data.selectorMtf;
1247 
1248     int bsLiveShadow = this.bsLive;
1249     int bsBuffShadow = this.bsBuff;
1250 
1251     for (int i = 0; i < nSelectors; i++) {
1252       for (int j = 0, hj = selectorMtf[i] & 0xff; j < hj; j++) {
1253         // inlined: bsW(1, 1);
1254         while (bsLiveShadow >= 8) {
1255           outShadow.write(bsBuffShadow >> 24);
1256           bsBuffShadow <<= 8;
1257           bsLiveShadow -= 8;
1258         }
1259         bsBuffShadow |= 1 << (32 - bsLiveShadow - 1);
1260         bsLiveShadow++;
1261       }
1262 
1263       // inlined: bsW(1, 0);
1264       while (bsLiveShadow >= 8) {
1265         outShadow.write(bsBuffShadow >> 24);
1266         bsBuffShadow <<= 8;
1267         bsLiveShadow -= 8;
1268       }
1269       // bsBuffShadow |= 0 << (32 - bsLiveShadow - 1);
1270       bsLiveShadow++;
1271     }
1272 
1273     this.bsBuff = bsBuffShadow;
1274     this.bsLive = bsLiveShadow;
1275   }
1276 
sendMTFValues6(final int nGroups, final int alphaSize)1277   private void sendMTFValues6(final int nGroups, final int alphaSize)
1278       throws IOException {
1279     final byte[][] len = this.data.sendMTFValues_len;
1280     final OutputStream outShadow = this.out;
1281 
1282     int bsLiveShadow = this.bsLive;
1283     int bsBuffShadow = this.bsBuff;
1284 
1285     for (int t = 0; t < nGroups; t++) {
1286       byte[] len_t = len[t];
1287       int curr = len_t[0] & 0xff;
1288 
1289       // inlined: bsW(5, curr);
1290       while (bsLiveShadow >= 8) {
1291         outShadow.write(bsBuffShadow >> 24); // write 8-bit
1292         bsBuffShadow <<= 8;
1293         bsLiveShadow -= 8;
1294       }
1295       bsBuffShadow |= curr << (32 - bsLiveShadow - 5);
1296       bsLiveShadow += 5;
1297 
1298       for (int i = 0; i < alphaSize; i++) {
1299         int lti = len_t[i] & 0xff;
1300         while (curr < lti) {
1301           // inlined: bsW(2, 2);
1302           while (bsLiveShadow >= 8) {
1303             outShadow.write(bsBuffShadow >> 24); // write 8-bit
1304             bsBuffShadow <<= 8;
1305             bsLiveShadow -= 8;
1306           }
1307           bsBuffShadow |= 2 << (32 - bsLiveShadow - 2);
1308           bsLiveShadow += 2;
1309 
1310           curr++; /* 10 */
1311         }
1312 
1313         while (curr > lti) {
1314           // inlined: bsW(2, 3);
1315           while (bsLiveShadow >= 8) {
1316             outShadow.write(bsBuffShadow >> 24); // write 8-bit
1317             bsBuffShadow <<= 8;
1318             bsLiveShadow -= 8;
1319           }
1320           bsBuffShadow |= 3 << (32 - bsLiveShadow - 2);
1321           bsLiveShadow += 2;
1322 
1323           curr--; /* 11 */
1324         }
1325 
1326         // inlined: bsW(1, 0);
1327         while (bsLiveShadow >= 8) {
1328           outShadow.write(bsBuffShadow >> 24); // write 8-bit
1329           bsBuffShadow <<= 8;
1330           bsLiveShadow -= 8;
1331         }
1332         // bsBuffShadow |= 0 << (32 - bsLiveShadow - 1);
1333         bsLiveShadow++;
1334       }
1335     }
1336 
1337     this.bsBuff = bsBuffShadow;
1338     this.bsLive = bsLiveShadow;
1339   }
1340 
sendMTFValues7(final int nSelectors)1341   private void sendMTFValues7(final int nSelectors) throws IOException {
1342     final Data dataShadow = this.data;
1343     final byte[][] len = dataShadow.sendMTFValues_len;
1344     final int[][] code = dataShadow.sendMTFValues_code;
1345     final OutputStream outShadow = this.out;
1346     final byte[] selector = dataShadow.selector;
1347     final char[] sfmap = dataShadow.sfmap;
1348     final int nMTFShadow = this.nMTF;
1349 
1350     int selCtr = 0;
1351 
1352     int bsLiveShadow = this.bsLive;
1353     int bsBuffShadow = this.bsBuff;
1354 
1355     for (int gs = 0; gs < nMTFShadow;) {
1356       final int ge = Math.min(gs + G_SIZE - 1, nMTFShadow - 1);
1357       final int selector_selCtr = selector[selCtr] & 0xff;
1358       final int[] code_selCtr = code[selector_selCtr];
1359       final byte[] len_selCtr = len[selector_selCtr];
1360 
1361       while (gs <= ge) {
1362         final int sfmap_i = sfmap[gs];
1363 
1364         //
1365         // inlined: bsW(len_selCtr[sfmap_i] & 0xff,
1366         // code_selCtr[sfmap_i]);
1367         //
1368         while (bsLiveShadow >= 8) {
1369           outShadow.write(bsBuffShadow >> 24);
1370           bsBuffShadow <<= 8;
1371           bsLiveShadow -= 8;
1372         }
1373         final int n = len_selCtr[sfmap_i] & 0xFF;
1374         bsBuffShadow |= code_selCtr[sfmap_i] << (32 - bsLiveShadow - n);
1375         bsLiveShadow += n;
1376 
1377         gs++;
1378       }
1379 
1380       gs = ge + 1;
1381       selCtr++;
1382     }
1383 
1384     this.bsBuff = bsBuffShadow;
1385     this.bsLive = bsLiveShadow;
1386   }
1387 
moveToFrontCodeAndSend()1388   private void moveToFrontCodeAndSend() throws IOException {
1389     bsW(24, this.origPtr);
1390     generateMTFValues();
1391     sendMTFValues();
1392   }
1393 
1394   /**
1395   * This is the most hammered method of this class.
1396   *
1397   * <p>
1398   * This is the version using unrolled loops. Normally I never use such ones
1399   * in Java code. The unrolling has shown a noticable performance improvement
1400   * on JRE 1.4.2 (Linux i586 / HotSpot Client). Of course it depends on the
1401   * JIT compiler of the vm.
1402   * </p>
1403   */
mainSimpleSort(final Data dataShadow, final int lo, final int hi, final int d)1404   private boolean mainSimpleSort(final Data dataShadow, final int lo,
1405       final int hi, final int d) {
1406     final int bigN = hi - lo + 1;
1407     if (bigN < 2) {
1408       return this.firstAttempt && (this.workDone > this.workLimit);
1409     }
1410 
1411     int hp = 0;
1412     while (INCS[hp] < bigN) {
1413       hp++;
1414     }
1415 
1416     final int[] fmap = dataShadow.fmap;
1417     final char[] quadrant = dataShadow.quadrant;
1418     final byte[] block = dataShadow.block;
1419     final int lastShadow = this.last;
1420     final int lastPlus1 = lastShadow + 1;
1421     final boolean firstAttemptShadow = this.firstAttempt;
1422     final int workLimitShadow = this.workLimit;
1423     int workDoneShadow = this.workDone;
1424 
1425     // Following block contains unrolled code which could be shortened by
1426     // coding it in additional loops.
1427 
1428     HP: while (--hp >= 0) {
1429       final int h = INCS[hp];
1430       final int mj = lo + h - 1;
1431 
1432       for (int i = lo + h; i <= hi;) {
1433         // copy
1434         for (int k = 3; (i <= hi) && (--k >= 0); i++) {
1435           final int v = fmap[i];
1436           final int vd = v + d;
1437           int j = i;
1438 
1439           // for (int a;
1440           // (j > mj) && mainGtU((a = fmap[j - h]) + d, vd,
1441           // block, quadrant, lastShadow);
1442           // j -= h) {
1443           // fmap[j] = a;
1444           // }
1445           //
1446           // unrolled version:
1447 
1448           // start inline mainGTU
1449           boolean onceRunned = false;
1450           int a = 0;
1451 
1452           HAMMER: while (true) {
1453             if (onceRunned) {
1454               fmap[j] = a;
1455               if ((j -= h) <= mj) {
1456                 break HAMMER;
1457               }
1458             } else {
1459               onceRunned = true;
1460             }
1461 
1462             a = fmap[j - h];
1463             int i1 = a + d;
1464             int i2 = vd;
1465 
1466             // following could be done in a loop, but
1467             // unrolled it for performance:
1468             if (block[i1 + 1] == block[i2 + 1]) {
1469               if (block[i1 + 2] == block[i2 + 2]) {
1470                 if (block[i1 + 3] == block[i2 + 3]) {
1471                   if (block[i1 + 4] == block[i2 + 4]) {
1472                     if (block[i1 + 5] == block[i2 + 5]) {
1473                       if (block[(i1 += 6)] == block[(i2 += 6)]) {
1474                         int x = lastShadow;
1475                         X: while (x > 0) {
1476                           x -= 4;
1477 
1478                           if (block[i1 + 1] == block[i2 + 1]) {
1479                             if (quadrant[i1] == quadrant[i2]) {
1480                               if (block[i1 + 2] == block[i2 + 2]) {
1481                                 if (quadrant[i1 + 1] == quadrant[i2 + 1]) {
1482                                   if (block[i1 + 3] == block[i2 + 3]) {
1483                                     if (quadrant[i1 + 2] == quadrant[i2 + 2]) {
1484                                       if (block[i1 + 4] == block[i2 + 4]) {
1485                                         if (quadrant[i1 + 3] == quadrant[i2 + 3]) {
1486                                           if ((i1 += 4) >= lastPlus1) {
1487                                             i1 -= lastPlus1;
1488                                           }
1489                                           if ((i2 += 4) >= lastPlus1) {
1490                                             i2 -= lastPlus1;
1491                                           }
1492                                           workDoneShadow++;
1493                                           continue X;
1494                                         } else if ((quadrant[i1 + 3] > quadrant[i2 + 3])) {
1495                                           continue HAMMER;
1496                                         } else {
1497                                           break HAMMER;
1498                                         }
1499                                       } else if ((block[i1 + 4] & 0xff) > (block[i2 + 4] & 0xff)) {
1500                                         continue HAMMER;
1501                                       } else {
1502                                         break HAMMER;
1503                                       }
1504                                     } else if ((quadrant[i1 + 2] > quadrant[i2 + 2])) {
1505                                       continue HAMMER;
1506                                     } else {
1507                                       break HAMMER;
1508                                     }
1509                                   } else if ((block[i1 + 3] & 0xff) > (block[i2 + 3] & 0xff)) {
1510                                     continue HAMMER;
1511                                   } else {
1512                                     break HAMMER;
1513                                   }
1514                                 } else if ((quadrant[i1 + 1] > quadrant[i2 + 1])) {
1515                                   continue HAMMER;
1516                                 } else {
1517                                   break HAMMER;
1518                                 }
1519                               } else if ((block[i1 + 2] & 0xff) > (block[i2 + 2] & 0xff)) {
1520                                 continue HAMMER;
1521                               } else {
1522                                 break HAMMER;
1523                               }
1524                             } else if ((quadrant[i1] > quadrant[i2])) {
1525                               continue HAMMER;
1526                             } else {
1527                               break HAMMER;
1528                             }
1529                           } else if ((block[i1 + 1] & 0xff) > (block[i2 + 1] & 0xff)) {
1530                             continue HAMMER;
1531                           } else {
1532                             break HAMMER;
1533                           }
1534 
1535                         }
1536                         break HAMMER;
1537                       } // while x > 0
1538                       else {
1539                         if ((block[i1] & 0xff) > (block[i2] & 0xff)) {
1540                           continue HAMMER;
1541                         } else {
1542                           break HAMMER;
1543                         }
1544                       }
1545                     } else if ((block[i1 + 5] & 0xff) > (block[i2 + 5] & 0xff)) {
1546                       continue HAMMER;
1547                     } else {
1548                       break HAMMER;
1549                     }
1550                   } else if ((block[i1 + 4] & 0xff) > (block[i2 + 4] & 0xff)) {
1551                     continue HAMMER;
1552                   } else {
1553                     break HAMMER;
1554                   }
1555                 } else if ((block[i1 + 3] & 0xff) > (block[i2 + 3] & 0xff)) {
1556                   continue HAMMER;
1557                 } else {
1558                   break HAMMER;
1559                 }
1560               } else if ((block[i1 + 2] & 0xff) > (block[i2 + 2] & 0xff)) {
1561                 continue HAMMER;
1562               } else {
1563                 break HAMMER;
1564               }
1565             } else if ((block[i1 + 1] & 0xff) > (block[i2 + 1] & 0xff)) {
1566               continue HAMMER;
1567             } else {
1568               break HAMMER;
1569             }
1570 
1571           } // HAMMER
1572           // end inline mainGTU
1573 
1574           fmap[j] = v;
1575         }
1576 
1577         if (firstAttemptShadow && (i <= hi)
1578             && (workDoneShadow > workLimitShadow)) {
1579           break HP;
1580         }
1581       }
1582     }
1583 
1584     this.workDone = workDoneShadow;
1585     return firstAttemptShadow && (workDoneShadow > workLimitShadow);
1586   }
1587 
vswap(int[] fmap, int p1, int p2, int n)1588   private static void vswap(int[] fmap, int p1, int p2, int n) {
1589     n += p1;
1590     while (p1 < n) {
1591       int t = fmap[p1];
1592       fmap[p1++] = fmap[p2];
1593       fmap[p2++] = t;
1594     }
1595   }
1596 
med3(byte a, byte b, byte c)1597   private static byte med3(byte a, byte b, byte c) {
1598     return (a < b) ? (b < c ? b : a < c ? c : a) : (b > c ? b : a > c ? c
1599         : a);
1600   }
1601 
blockSort()1602   private void blockSort() {
1603     this.workLimit = WORK_FACTOR * this.last;
1604     this.workDone = 0;
1605     this.blockRandomised = false;
1606     this.firstAttempt = true;
1607     mainSort();
1608 
1609     if (this.firstAttempt && (this.workDone > this.workLimit)) {
1610       randomiseBlock();
1611       this.workLimit = this.workDone = 0;
1612       this.firstAttempt = false;
1613       mainSort();
1614     }
1615 
1616     int[] fmap = this.data.fmap;
1617     this.origPtr = -1;
1618     for (int i = 0, lastShadow = this.last; i <= lastShadow; i++) {
1619       if (fmap[i] == 0) {
1620         this.origPtr = i;
1621         break;
1622       }
1623     }
1624 
1625     // assert (this.origPtr != -1) : this.origPtr;
1626   }
1627 
1628   /**
1629   * Method "mainQSort3", file "blocksort.c", BZip2 1.0.2
1630   */
mainQSort3(final Data dataShadow, final int loSt, final int hiSt, final int dSt)1631   private void mainQSort3(final Data dataShadow, final int loSt,
1632       final int hiSt, final int dSt) {
1633     final int[] stack_ll = dataShadow.stack_ll;
1634     final int[] stack_hh = dataShadow.stack_hh;
1635     final int[] stack_dd = dataShadow.stack_dd;
1636     final int[] fmap = dataShadow.fmap;
1637     final byte[] block = dataShadow.block;
1638 
1639     stack_ll[0] = loSt;
1640     stack_hh[0] = hiSt;
1641     stack_dd[0] = dSt;
1642 
1643     for (int sp = 1; --sp >= 0;) {
1644       final int lo = stack_ll[sp];
1645       final int hi = stack_hh[sp];
1646       final int d = stack_dd[sp];
1647 
1648       if ((hi - lo < SMALL_THRESH) || (d > DEPTH_THRESH)) {
1649         if (mainSimpleSort(dataShadow, lo, hi, d)) {
1650           return;
1651         }
1652       } else {
1653         final int d1 = d + 1;
1654         final int med = med3(block[fmap[lo] + d1],
1655             block[fmap[hi] + d1], block[fmap[(lo + hi) >>> 1] + d1]) & 0xff;
1656 
1657         int unLo = lo;
1658         int unHi = hi;
1659         int ltLo = lo;
1660         int gtHi = hi;
1661 
1662         while (true) {
1663           while (unLo <= unHi) {
1664             final int n = ((int) block[fmap[unLo] + d1] & 0xff)
1665                 - med;
1666             if (n == 0) {
1667               final int temp = fmap[unLo];
1668               fmap[unLo++] = fmap[ltLo];
1669               fmap[ltLo++] = temp;
1670             } else if (n < 0) {
1671               unLo++;
1672             } else {
1673               break;
1674             }
1675           }
1676 
1677           while (unLo <= unHi) {
1678             final int n = ((int) block[fmap[unHi] + d1] & 0xff)
1679                 - med;
1680             if (n == 0) {
1681               final int temp = fmap[unHi];
1682               fmap[unHi--] = fmap[gtHi];
1683               fmap[gtHi--] = temp;
1684             } else if (n > 0) {
1685               unHi--;
1686             } else {
1687               break;
1688             }
1689           }
1690 
1691           if (unLo <= unHi) {
1692             final int temp = fmap[unLo];
1693             fmap[unLo++] = fmap[unHi];
1694             fmap[unHi--] = temp;
1695           } else {
1696             break;
1697           }
1698         }
1699 
1700         if (gtHi < ltLo) {
1701           stack_ll[sp] = lo;
1702           stack_hh[sp] = hi;
1703           stack_dd[sp] = d1;
1704           sp++;
1705         } else {
1706           int n = ((ltLo - lo) < (unLo - ltLo)) ? (ltLo - lo)
1707               : (unLo - ltLo);
1708           vswap(fmap, lo, unLo - n, n);
1709           int m = ((hi - gtHi) < (gtHi - unHi)) ? (hi - gtHi)
1710               : (gtHi - unHi);
1711           vswap(fmap, unLo, hi - m + 1, m);
1712 
1713           n = lo + unLo - ltLo - 1;
1714           m = hi - (gtHi - unHi) + 1;
1715 
1716           stack_ll[sp] = lo;
1717           stack_hh[sp] = n;
1718           stack_dd[sp] = d;
1719           sp++;
1720 
1721           stack_ll[sp] = n + 1;
1722           stack_hh[sp] = m - 1;
1723           stack_dd[sp] = d1;
1724           sp++;
1725 
1726           stack_ll[sp] = m;
1727           stack_hh[sp] = hi;
1728           stack_dd[sp] = d;
1729           sp++;
1730         }
1731       }
1732     }
1733   }
1734 
mainSort()1735   private void mainSort() {
1736     final Data dataShadow = this.data;
1737     final int[] runningOrder = dataShadow.mainSort_runningOrder;
1738     final int[] copy = dataShadow.mainSort_copy;
1739     final boolean[] bigDone = dataShadow.mainSort_bigDone;
1740     final int[] ftab = dataShadow.ftab;
1741     final byte[] block = dataShadow.block;
1742     final int[] fmap = dataShadow.fmap;
1743     final char[] quadrant = dataShadow.quadrant;
1744     final int lastShadow = this.last;
1745     final int workLimitShadow = this.workLimit;
1746     final boolean firstAttemptShadow = this.firstAttempt;
1747 
1748     // Set up the 2-byte frequency table
1749     for (int i = 65537; --i >= 0;) {
1750       ftab[i] = 0;
1751     }
1752 
1753     /*
1754     * In the various block-sized structures, live data runs from 0 to
1755     * last+NUM_OVERSHOOT_BYTES inclusive. First, set up the overshoot area
1756     * for block.
1757     */
1758     for (int i = 0; i < NUM_OVERSHOOT_BYTES; i++) {
1759       block[lastShadow + i + 2] = block[(i % (lastShadow + 1)) + 1];
1760     }
1761     for (int i = lastShadow + NUM_OVERSHOOT_BYTES +1; --i >= 0;) {
1762       quadrant[i] = 0;
1763     }
1764     block[0] = block[lastShadow + 1];
1765 
1766     // Complete the initial radix sort:
1767 
1768     int c1 = block[0] & 0xff;
1769     for (int i = 0; i <= lastShadow; i++) {
1770       final int c2 = block[i + 1] & 0xff;
1771       ftab[(c1 << 8) + c2]++;
1772       c1 = c2;
1773     }
1774 
1775     for (int i = 1; i <= 65536; i++)
1776       ftab[i] += ftab[i - 1];
1777 
1778     c1 = block[1] & 0xff;
1779     for (int i = 0; i < lastShadow; i++) {
1780       final int c2 = block[i + 2] & 0xff;
1781       fmap[--ftab[(c1 << 8) + c2]] = i;
1782       c1 = c2;
1783     }
1784 
1785     fmap[--ftab[((block[lastShadow + 1] & 0xff) << 8) + (block[1] & 0xff)]] = lastShadow;
1786 
1787     /*
1788     * Now ftab contains the first loc of every small bucket. Calculate the
1789     * running order, from smallest to largest big bucket.
1790     */
1791     for (int i = 256; --i >= 0;) {
1792       bigDone[i] = false;
1793       runningOrder[i] = i;
1794     }
1795 
1796     for (int h = 364; h != 1;) {
1797       h /= 3;
1798       for (int i = h; i <= 255; i++) {
1799         final int vv = runningOrder[i];
1800         final int a = ftab[(vv + 1) << 8] - ftab[vv << 8];
1801         final int b = h - 1;
1802         int j = i;
1803         for (int ro = runningOrder[j - h]; (ftab[(ro + 1) << 8] - ftab[ro << 8]) > a; ro = runningOrder[j
1804             - h]) {
1805           runningOrder[j] = ro;
1806           j -= h;
1807           if (j <= b) {
1808             break;
1809           }
1810         }
1811         runningOrder[j] = vv;
1812       }
1813     }
1814 
1815     /*
1816     * The main sorting loop.
1817     */
1818     for (int i = 0; i <= 255; i++) {
1819       /*
1820       * Process big buckets, starting with the least full.
1821       */
1822       final int ss = runningOrder[i];
1823 
1824       // Step 1:
1825       /*
1826       * Complete the big bucket [ss] by quicksorting any unsorted small
1827       * buckets [ss, j]. Hopefully previous pointer-scanning phases have
1828       * already completed many of the small buckets [ss, j], so we don't
1829       * have to sort them at all.
1830       */
1831       for (int j = 0; j <= 255; j++) {
1832         final int sb = (ss << 8) + j;
1833         final int ftab_sb = ftab[sb];
1834         if ((ftab_sb & SETMASK) != SETMASK) {
1835           final int lo = ftab_sb & CLEARMASK;
1836           final int hi = (ftab[sb + 1] & CLEARMASK) - 1;
1837           if (hi > lo) {
1838             mainQSort3(dataShadow, lo, hi, 2);
1839             if (firstAttemptShadow
1840                 && (this.workDone > workLimitShadow)) {
1841               return;
1842             }
1843           }
1844           ftab[sb] = ftab_sb | SETMASK;
1845         }
1846       }
1847 
1848       // Step 2:
1849       // Now scan this big bucket so as to synthesise the
1850       // sorted order for small buckets [t, ss] for all t != ss.
1851 
1852       for (int j = 0; j <= 255; j++) {
1853         copy[j] = ftab[(j << 8) + ss] & CLEARMASK;
1854       }
1855 
1856       for (int j = ftab[ss << 8] & CLEARMASK, hj = (ftab[(ss + 1) << 8] & CLEARMASK); j < hj; j++) {
1857         final int fmap_j = fmap[j];
1858         c1 = block[fmap_j] & 0xff;
1859         if (!bigDone[c1]) {
1860           fmap[copy[c1]] = (fmap_j == 0) ? lastShadow : (fmap_j - 1);
1861           copy[c1]++;
1862         }
1863       }
1864 
1865       for (int j = 256; --j >= 0;)
1866         ftab[(j << 8) + ss] |= SETMASK;
1867 
1868       // Step 3:
1869       /*
1870       * The ss big bucket is now done. Record this fact, and update the
1871       * quadrant descriptors. Remember to update quadrants in the
1872       * overshoot area too, if necessary. The "if (i < 255)" test merely
1873       * skips this updating for the last bucket processed, since updating
1874       * for the last bucket is pointless.
1875       */
1876       bigDone[ss] = true;
1877 
1878       if (i < 255) {
1879         final int bbStart = ftab[ss << 8] & CLEARMASK;
1880         final int bbSize = (ftab[(ss + 1) << 8] & CLEARMASK) - bbStart;
1881         int shifts = 0;
1882 
1883         while ((bbSize >> shifts) > 65534) {
1884           shifts++;
1885         }
1886 
1887         for (int j = 0; j < bbSize; j++) {
1888           final int a2update = fmap[bbStart + j];
1889           final char qVal = (char) (j >> shifts);
1890           quadrant[a2update] = qVal;
1891           if (a2update < NUM_OVERSHOOT_BYTES) {
1892             quadrant[a2update + lastShadow + 1] = qVal;
1893           }
1894         }
1895       }
1896 
1897     }
1898   }
1899 
randomiseBlock()1900   private void randomiseBlock() {
1901     final boolean[] inUse = this.data.inUse;
1902     final byte[] block = this.data.block;
1903     final int lastShadow = this.last;
1904 
1905     for (int i = 256; --i >= 0;)
1906       inUse[i] = false;
1907 
1908     int rNToGo = 0;
1909     int rTPos = 0;
1910     for (int i = 0, j = 1; i <= lastShadow; i = j, j++) {
1911       if (rNToGo == 0) {
1912         rNToGo = (char) BZip2Constants.rNums[rTPos];
1913         if (++rTPos == 512) {
1914           rTPos = 0;
1915         }
1916       }
1917 
1918       rNToGo--;
1919       block[j] ^= ((rNToGo == 1) ? 1 : 0);
1920 
1921       // handle 16 bit signed numbers
1922       inUse[block[j] & 0xff] = true;
1923     }
1924 
1925     this.blockRandomised = true;
1926   }
1927 
generateMTFValues()1928   private void generateMTFValues() {
1929     final int lastShadow = this.last;
1930     final Data dataShadow = this.data;
1931     final boolean[] inUse = dataShadow.inUse;
1932     final byte[] block = dataShadow.block;
1933     final int[] fmap = dataShadow.fmap;
1934     final char[] sfmap = dataShadow.sfmap;
1935     final int[] mtfFreq = dataShadow.mtfFreq;
1936     final byte[] unseqToSeq = dataShadow.unseqToSeq;
1937     final byte[] yy = dataShadow.generateMTFValues_yy;
1938 
1939     // make maps
1940     int nInUseShadow = 0;
1941     for (int i = 0; i < 256; i++) {
1942       if (inUse[i]) {
1943         unseqToSeq[i] = (byte) nInUseShadow;
1944         nInUseShadow++;
1945       }
1946     }
1947     this.nInUse = nInUseShadow;
1948 
1949     final int eob = nInUseShadow + 1;
1950 
1951     for (int i = eob; i >= 0; i--) {
1952       mtfFreq[i] = 0;
1953     }
1954 
1955     for (int i = nInUseShadow; --i >= 0;) {
1956       yy[i] = (byte) i;
1957     }
1958 
1959     int wr = 0;
1960     int zPend = 0;
1961 
1962     for (int i = 0; i <= lastShadow; i++) {
1963       final byte ll_i = unseqToSeq[block[fmap[i]] & 0xff];
1964       byte tmp = yy[0];
1965       int j = 0;
1966 
1967       while (ll_i != tmp) {
1968         j++;
1969         byte tmp2 = tmp;
1970         tmp = yy[j];
1971         yy[j] = tmp2;
1972       }
1973       yy[0] = tmp;
1974 
1975       if (j == 0) {
1976         zPend++;
1977       } else {
1978         if (zPend > 0) {
1979           zPend--;
1980           while (true) {
1981             if ((zPend & 1) == 0) {
1982               sfmap[wr] = RUNA;
1983               wr++;
1984               mtfFreq[RUNA]++;
1985             } else {
1986               sfmap[wr] = RUNB;
1987               wr++;
1988               mtfFreq[RUNB]++;
1989             }
1990 
1991             if (zPend >= 2) {
1992               zPend = (zPend - 2) >> 1;
1993             } else {
1994               break;
1995             }
1996           }
1997           zPend = 0;
1998         }
1999         sfmap[wr] = (char) (j + 1);
2000         wr++;
2001         mtfFreq[j + 1]++;
2002       }
2003     }
2004 
2005     if (zPend > 0) {
2006       zPend--;
2007       while (true) {
2008         if ((zPend & 1) == 0) {
2009           sfmap[wr] = RUNA;
2010           wr++;
2011           mtfFreq[RUNA]++;
2012         } else {
2013           sfmap[wr] = RUNB;
2014           wr++;
2015           mtfFreq[RUNB]++;
2016         }
2017 
2018         if (zPend >= 2) {
2019           zPend = (zPend - 2) >> 1;
2020         } else {
2021           break;
2022         }
2023       }
2024     }
2025 
2026     sfmap[wr] = (char) eob;
2027     mtfFreq[eob]++;
2028     this.nMTF = wr + 1;
2029   }
2030 
2031   private static final class Data extends Object {
2032 
2033     // with blockSize 900k
2034     final boolean[] inUse = new boolean[256]; // 256 byte
2035     final byte[] unseqToSeq = new byte[256]; // 256 byte
2036     final int[] mtfFreq = new int[MAX_ALPHA_SIZE]; // 1032 byte
2037     final byte[] selector = new byte[MAX_SELECTORS]; // 18002 byte
2038     final byte[] selectorMtf = new byte[MAX_SELECTORS]; // 18002 byte
2039 
2040     final byte[] generateMTFValues_yy = new byte[256]; // 256 byte
2041     final byte[][] sendMTFValues_len = new byte[N_GROUPS][MAX_ALPHA_SIZE]; // 1548
2042     // byte
2043     final int[][] sendMTFValues_rfreq = new int[N_GROUPS][MAX_ALPHA_SIZE]; // 6192
2044     // byte
2045     final int[] sendMTFValues_fave = new int[N_GROUPS]; // 24 byte
2046     final short[] sendMTFValues_cost = new short[N_GROUPS]; // 12 byte
2047     final int[][] sendMTFValues_code = new int[N_GROUPS][MAX_ALPHA_SIZE]; // 6192
2048     // byte
2049     final byte[] sendMTFValues2_pos = new byte[N_GROUPS]; // 6 byte
2050     final boolean[] sentMTFValues4_inUse16 = new boolean[16]; // 16 byte
2051 
2052     final int[] stack_ll = new int[QSORT_STACK_SIZE]; // 4000 byte
2053     final int[] stack_hh = new int[QSORT_STACK_SIZE]; // 4000 byte
2054     final int[] stack_dd = new int[QSORT_STACK_SIZE]; // 4000 byte
2055 
2056     final int[] mainSort_runningOrder = new int[256]; // 1024 byte
2057     final int[] mainSort_copy = new int[256]; // 1024 byte
2058     final boolean[] mainSort_bigDone = new boolean[256]; // 256 byte
2059 
2060     final int[] heap = new int[MAX_ALPHA_SIZE + 2]; // 1040 byte
2061     final int[] weight = new int[MAX_ALPHA_SIZE * 2]; // 2064 byte
2062     final int[] parent = new int[MAX_ALPHA_SIZE * 2]; // 2064 byte
2063 
2064     final int[] ftab = new int[65537]; // 262148 byte
2065     // ------------
2066     // 333408 byte
2067 
2068     final byte[] block; // 900021 byte
2069     final int[] fmap; // 3600000 byte
2070     final char[] sfmap; // 3600000 byte
2071     // ------------
2072     // 8433529 byte
2073     // ============
2074 
2075     /**
2076     * Array instance identical to sfmap, both are used only temporarily and
2077     * indepently, so we do not need to allocate additional memory.
2078     */
2079     final char[] quadrant;
2080 
Data(int blockSize100k)2081     Data(int blockSize100k) {
2082       super();
2083 
2084       final int n = blockSize100k * BZip2Constants.baseBlockSize;
2085       this.block = new byte[(n + 1 + NUM_OVERSHOOT_BYTES)];
2086       this.fmap = new int[n];
2087       this.sfmap = new char[2 * n];
2088       this.quadrant = this.sfmap;
2089     }
2090 
2091   }
2092 
2093 }
2094