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+ package org .apache .hadoop .io .compress ;
19+
20+ import java .io .IOException ;
21+ import java .io .InputStream ;
22+ import java .util .List ;
23+
24+ import org .apache .hadoop .thirdparty .com .google .common .primitives .Bytes ;
25+ import org .junit .After ;
26+ import org .junit .Before ;
27+ import org .junit .Test ;
28+
29+ import org .apache .commons .io .IOUtils ;
30+ import org .apache .hadoop .conf .Configuration ;
31+ import org .apache .hadoop .fs .FSDataInputStream ;
32+ import org .apache .hadoop .fs .FileSystem ;
33+ import org .apache .hadoop .fs .Path ;
34+ import org .apache .hadoop .io .compress .SplittableCompressionCodec .READ_MODE ;
35+ import org .apache .hadoop .io .compress .bzip2 .BZip2TextFileWriter ;
36+ import org .apache .hadoop .io .compress .bzip2 .BZip2Utils ;
37+
38+ import static org .apache .hadoop .io .compress .SplittableCompressionCodec .READ_MODE .BYBLOCK ;
39+ import static org .apache .hadoop .io .compress .SplittableCompressionCodec .READ_MODE .CONTINUOUS ;
40+ import static org .apache .hadoop .io .compress .bzip2 .BZip2TextFileWriter .BLOCK_SIZE ;
41+ import static org .apache .hadoop .util .Preconditions .checkArgument ;
42+ import static org .assertj .core .api .Assertions .assertThatNullPointerException ;
43+ import static org .assertj .core .api .AssertionsForClassTypes .assertThatExceptionOfType ;
44+ import static org .junit .Assert .assertArrayEquals ;
45+ import static org .junit .Assert .assertEquals ;
46+
47+ public final class TestBZip2Codec {
48+
49+ private static final long HEADER_LEN = 2 ;
50+
51+ private Configuration conf ;
52+ private FileSystem fs ;
53+ private BZip2Codec codec ;
54+ private Decompressor decompressor ;
55+ private Path tempFile ;
56+
57+ @ Before
58+ public void setUp () throws Exception {
59+ conf = new Configuration ();
60+
61+ Path workDir = new Path (System .getProperty ("test.build.data" , "target" ),
62+ "data/" + getClass ().getSimpleName ());
63+
64+ Path inputDir = new Path (workDir , "input" );
65+ tempFile = new Path (inputDir , "test.txt.bz2" );
66+
67+ fs = workDir .getFileSystem (conf );
68+
69+ codec = new BZip2Codec ();
70+ codec .setConf (new Configuration (/* loadDefaults */ false ));
71+ decompressor = CodecPool .getDecompressor (codec );
72+ }
73+
74+ @ After
75+ public void tearDown () throws Exception {
76+ CodecPool .returnDecompressor (decompressor );
77+ fs .delete (tempFile , /* recursive */ false );
78+ }
79+
80+ @ Test
81+ public void createInputStreamWithStartAndEnd () throws Exception {
82+ byte [] data1 = newAlternatingByteArray (BLOCK_SIZE , 'a' , 'b' );
83+ byte [] data2 = newAlternatingByteArray (BLOCK_SIZE , 'c' , 'd' );
84+ byte [] data3 = newAlternatingByteArray (BLOCK_SIZE , 'e' , 'f' );
85+
86+ try (BZip2TextFileWriter writer = new BZip2TextFileWriter (tempFile , conf )) {
87+ writer .write (data1 );
88+ writer .write (data2 );
89+ writer .write (data3 );
90+ }
91+ long fileSize = fs .getFileStatus (tempFile ).getLen ();
92+
93+ List <Long > nextBlockOffsets = BZip2Utils .getNextBlockMarkerOffsets (tempFile , conf );
94+ long block2Start = nextBlockOffsets .get (0 );
95+ long block3Start = nextBlockOffsets .get (1 );
96+
97+ try (SplitCompressionInputStream stream = newCompressionStream (tempFile , 0 , fileSize ,
98+ BYBLOCK )) {
99+ assertEquals (0 , stream .getPos ());
100+ assertCasesWhereReadDoesNotAdvanceStream (stream );
101+ assertReadingAtPositionZero (stream , data1 );
102+ assertCasesWhereReadDoesNotAdvanceStream (stream );
103+ assertReadingPastEndOfBlock (stream , block2Start , data2 );
104+ assertReadingPastEndOfBlock (stream , block3Start , data3 );
105+ assertEquals (-1 , stream .read ());
106+ }
107+
108+ try (SplitCompressionInputStream stream = newCompressionStream (tempFile , 1 , fileSize - 1 ,
109+ BYBLOCK )) {
110+ assertEquals (block2Start , stream .getPos ());
111+ assertCasesWhereReadDoesNotAdvanceStream (stream );
112+ assertReadingPastEndOfBlock (stream , block2Start , data2 );
113+ assertCasesWhereReadDoesNotAdvanceStream (stream );
114+ assertReadingPastEndOfBlock (stream , block3Start , data3 );
115+ assertEquals (-1 , stream .read ());
116+ }
117+
118+ // With continuous mode, only starting at or after the stream header is
119+ // supported.
120+ byte [] allData = Bytes .concat (data1 , data2 , data3 );
121+ assertReadingWithContinuousMode (tempFile , 0 , fileSize , allData );
122+ assertReadingWithContinuousMode (tempFile , HEADER_LEN , fileSize - HEADER_LEN , allData );
123+ }
124+
125+ private void assertReadingWithContinuousMode (Path file , long start , long length ,
126+ byte [] expectedData ) throws IOException {
127+ try (SplitCompressionInputStream stream = newCompressionStream (file , start , length ,
128+ CONTINUOUS )) {
129+ assertEquals (HEADER_LEN , stream .getPos ());
130+
131+ assertRead (stream , expectedData );
132+ assertEquals (-1 , stream .read ());
133+
134+ // When specifying CONTINUOUS read mode, the position ends up not being
135+ // updated at all.
136+ assertEquals (HEADER_LEN , stream .getPos ());
137+ }
138+ }
139+
140+ private SplitCompressionInputStream newCompressionStream (Path file , long start , long length ,
141+ READ_MODE readMode ) throws IOException {
142+ FSDataInputStream rawIn = fs .open (file );
143+ rawIn .seek (start );
144+ long end = start + length ;
145+ return codec .createInputStream (rawIn , decompressor , start , end , readMode );
146+ }
147+
148+ private static byte [] newAlternatingByteArray (int size , int ... choices ) {
149+ checkArgument (choices .length > 1 );
150+ byte [] result = new byte [size ];
151+ for (int i = 0 ; i < size ; i ++) {
152+ result [i ] = (byte ) choices [i % choices .length ];
153+ }
154+ return result ;
155+ }
156+
157+ private static void assertCasesWhereReadDoesNotAdvanceStream (SplitCompressionInputStream in )
158+ throws IOException {
159+ long initialPos = in .getPos ();
160+
161+ assertEquals (0 , in .read (new byte [0 ]));
162+
163+ assertThatNullPointerException ().isThrownBy (() -> in .read (null , 0 , 1 ));
164+ assertThatExceptionOfType (IndexOutOfBoundsException .class ).isThrownBy (
165+ () -> in .read (new byte [5 ], -1 , 2 ));
166+ assertThatExceptionOfType (IndexOutOfBoundsException .class ).isThrownBy (
167+ () -> in .read (new byte [5 ], 0 , -1 ));
168+ assertThatExceptionOfType (IndexOutOfBoundsException .class ).isThrownBy (
169+ () -> in .read (new byte [5 ], 1 , 5 ));
170+
171+ assertEquals (initialPos , in .getPos ());
172+ }
173+
174+ private static void assertReadingAtPositionZero (SplitCompressionInputStream in ,
175+ byte [] expectedData ) throws IOException {
176+ byte [] buffer = new byte [expectedData .length ];
177+ assertEquals (1 , in .read (buffer , 0 , 1 ));
178+ assertEquals (expectedData [0 ], buffer [0 ]);
179+ assertEquals (0 , in .getPos ());
180+
181+ IOUtils .readFully (in , buffer , 1 , expectedData .length - 1 );
182+ assertArrayEquals (expectedData , buffer );
183+ assertEquals (0 , in .getPos ());
184+ }
185+
186+ private static void assertReadingPastEndOfBlock (SplitCompressionInputStream in ,
187+ long endOfBlockPos , byte [] expectedData ) throws IOException {
188+ byte [] buffer = new byte [expectedData .length ];
189+ assertEquals (1 , in .read (buffer ));
190+ assertEquals (expectedData [0 ], buffer [0 ]);
191+ assertEquals (endOfBlockPos + 1 , in .getPos ());
192+
193+ IOUtils .readFully (in , buffer , 1 , expectedData .length - 1 );
194+ assertArrayEquals (expectedData , buffer );
195+ assertEquals (endOfBlockPos + 1 , in .getPos ());
196+ }
197+
198+ private static void assertRead (InputStream in , byte [] expectedData ) throws IOException {
199+ byte [] buffer = new byte [expectedData .length ];
200+ IOUtils .readFully (in , buffer );
201+ assertArrayEquals (expectedData , buffer );
202+ }
203+ }
0 commit comments