1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.io.compress;
18
19 import java.io.BufferedInputStream;
20 import java.io.BufferedOutputStream;
21 import java.io.FilterOutputStream;
22 import java.io.IOException;
23 import java.io.InputStream;
24 import java.io.OutputStream;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configurable;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.io.IOUtils;
32 import org.apache.hadoop.io.compress.CodecPool;
33 import org.apache.hadoop.io.compress.CompressionCodec;
34 import org.apache.hadoop.io.compress.CompressionInputStream;
35 import org.apache.hadoop.io.compress.CompressionOutputStream;
36 import org.apache.hadoop.io.compress.Compressor;
37 import org.apache.hadoop.io.compress.Decompressor;
38 import org.apache.hadoop.io.compress.DefaultCodec;
39 import org.apache.hadoop.io.compress.DoNotPool;
40 import org.apache.hadoop.io.compress.GzipCodec;
41 import org.apache.hadoop.util.ReflectionUtils;
42
43
44
45
46
47 @InterfaceAudience.Private
48 public final class Compression {
49 static final Log LOG = LogFactory.getLog(Compression.class);
50
51
52
53
54 private Compression() {
55 super();
56 }
57
58 static class FinishOnFlushCompressionStream extends FilterOutputStream {
59 public FinishOnFlushCompressionStream(CompressionOutputStream cout) {
60 super(cout);
61 }
62
63 @Override
64 public void write(byte b[], int off, int len) throws IOException {
65 out.write(b, off, len);
66 }
67
68 @Override
69 public void flush() throws IOException {
70 CompressionOutputStream cout = (CompressionOutputStream) out;
71 cout.finish();
72 cout.flush();
73 cout.resetState();
74 }
75 }
76
77
78
79
80
81 private static ClassLoader getClassLoaderForCodec() {
82 ClassLoader cl = Thread.currentThread().getContextClassLoader();
83 if (cl == null) {
84 cl = Compression.class.getClassLoader();
85 }
86 if (cl == null) {
87 cl = ClassLoader.getSystemClassLoader();
88 }
89 if (cl == null) {
90 throw new RuntimeException("A ClassLoader to load the Codec could not be determined");
91 }
92 return cl;
93 }
94
95
96
97
98
99
100 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
101 value="SE_TRANSIENT_FIELD_NOT_RESTORED",
102 justification="We are not serializing so doesn't apply (not sure why transient though)")
103 public static enum Algorithm {
104 LZO("lzo") {
105
106 private volatile transient CompressionCodec lzoCodec;
107 private transient Object lock = new Object();
108
109 @Override
110 CompressionCodec getCodec(Configuration conf) {
111 if (lzoCodec == null) {
112 synchronized (lock) {
113 if (lzoCodec == null) {
114 lzoCodec = buildCodec(conf);
115 }
116 }
117 }
118 return lzoCodec;
119 }
120
121 private CompressionCodec buildCodec(Configuration conf) {
122 try {
123 Class<?> externalCodec =
124 ClassLoader.getSystemClassLoader()
125 .loadClass("com.hadoop.compression.lzo.LzoCodec");
126 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec,
127 new Configuration(conf));
128 } catch (ClassNotFoundException e) {
129 throw new RuntimeException(e);
130 }
131 }
132 },
133 GZ("gz") {
134 private volatile transient GzipCodec codec;
135 private transient Object lock = new Object();
136
137 @Override
138 DefaultCodec getCodec(Configuration conf) {
139 if (codec == null) {
140 synchronized (lock) {
141 if (codec == null) {
142 codec = buildCodec(conf);
143 }
144 }
145 }
146
147 return codec;
148 }
149
150 private GzipCodec buildCodec(Configuration conf) {
151 GzipCodec gzcodec = new ReusableStreamGzipCodec();
152 gzcodec.setConf(new Configuration(conf));
153 return gzcodec;
154 }
155 },
156
157 NONE("none") {
158 @Override
159 DefaultCodec getCodec(Configuration conf) {
160 return null;
161 }
162
163 @Override
164 public synchronized InputStream createDecompressionStream(
165 InputStream downStream, Decompressor decompressor,
166 int downStreamBufferSize) throws IOException {
167 if (downStreamBufferSize > 0) {
168 return new BufferedInputStream(downStream, downStreamBufferSize);
169 }
170
171
172
173
174
175 return downStream;
176 }
177
178 @Override
179 public synchronized OutputStream createCompressionStream(
180 OutputStream downStream, Compressor compressor,
181 int downStreamBufferSize) throws IOException {
182 if (downStreamBufferSize > 0) {
183 return new BufferedOutputStream(downStream, downStreamBufferSize);
184 }
185
186 return downStream;
187 }
188 },
189 SNAPPY("snappy") {
190
191 private volatile transient CompressionCodec snappyCodec;
192 private transient Object lock = new Object();
193
194 @Override
195 CompressionCodec getCodec(Configuration conf) {
196 if (snappyCodec == null) {
197 synchronized (lock) {
198 if (snappyCodec == null) {
199 snappyCodec = buildCodec(conf);
200 }
201 }
202 }
203 return snappyCodec;
204 }
205
206 private CompressionCodec buildCodec(Configuration conf) {
207 try {
208 Class<?> externalCodec =
209 ClassLoader.getSystemClassLoader()
210 .loadClass("org.apache.hadoop.io.compress.SnappyCodec");
211 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec,
212 conf);
213 } catch (ClassNotFoundException e) {
214 throw new RuntimeException(e);
215 }
216 }
217 },
218 LZ4("lz4") {
219
220 private volatile transient CompressionCodec lz4Codec;
221 private transient Object lock = new Object();
222
223 @Override
224 CompressionCodec getCodec(Configuration conf) {
225 if (lz4Codec == null) {
226 synchronized (lock) {
227 if (lz4Codec == null) {
228 lz4Codec = buildCodec(conf);
229 }
230 }
231 buildCodec(conf);
232 }
233 return lz4Codec;
234 }
235
236 private CompressionCodec buildCodec(Configuration conf) {
237 try {
238 Class<?> externalCodec =
239 getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.Lz4Codec");
240 return (CompressionCodec) ReflectionUtils.newInstance(externalCodec,
241 conf);
242 } catch (ClassNotFoundException e) {
243 throw new RuntimeException(e);
244 }
245 }
246 };
247
248 private final Configuration conf;
249 private final String compressName;
250
251 private static final int DATA_IBUF_SIZE = 1 * 1024;
252
253 private static final int DATA_OBUF_SIZE = 4 * 1024;
254
255 Algorithm(String name) {
256 this.conf = new Configuration();
257 this.conf.setBoolean("hadoop.native.lib", true);
258 this.compressName = name;
259 }
260
261 abstract CompressionCodec getCodec(Configuration conf);
262
263 public InputStream createDecompressionStream(
264 InputStream downStream, Decompressor decompressor,
265 int downStreamBufferSize) throws IOException {
266 CompressionCodec codec = getCodec(conf);
267
268 if (downStreamBufferSize > 0) {
269 ((Configurable)codec).getConf().setInt("io.file.buffer.size",
270 downStreamBufferSize);
271 }
272 CompressionInputStream cis =
273 codec.createInputStream(downStream, decompressor);
274 BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
275 return bis2;
276
277 }
278
279 public OutputStream createCompressionStream(
280 OutputStream downStream, Compressor compressor, int downStreamBufferSize)
281 throws IOException {
282 OutputStream bos1 = null;
283 if (downStreamBufferSize > 0) {
284 bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
285 }
286 else {
287 bos1 = downStream;
288 }
289 CompressionOutputStream cos =
290 createPlainCompressionStream(bos1, compressor);
291 BufferedOutputStream bos2 =
292 new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
293 DATA_OBUF_SIZE);
294 return bos2;
295 }
296
297
298
299
300
301 public CompressionOutputStream createPlainCompressionStream(
302 OutputStream downStream, Compressor compressor) throws IOException {
303 CompressionCodec codec = getCodec(conf);
304 ((Configurable)codec).getConf().setInt("io.file.buffer.size", 32 * 1024);
305 return codec.createOutputStream(downStream, compressor);
306 }
307
308 public Compressor getCompressor() {
309 CompressionCodec codec = getCodec(conf);
310 if (codec != null) {
311 Compressor compressor = CodecPool.getCompressor(codec);
312 if (compressor != null) {
313 if (compressor.finished()) {
314
315
316 LOG
317 .warn("Compressor obtained from CodecPool is already finished()");
318
319
320 }
321 compressor.reset();
322 }
323 return compressor;
324 }
325 return null;
326 }
327
328 public void returnCompressor(Compressor compressor) {
329 if (compressor != null) {
330 CodecPool.returnCompressor(compressor);
331 }
332 }
333
334 public Decompressor getDecompressor() {
335 CompressionCodec codec = getCodec(conf);
336 if (codec != null) {
337 Decompressor decompressor = CodecPool.getDecompressor(codec);
338 if (decompressor != null) {
339 if (decompressor.finished()) {
340
341
342 LOG
343 .warn("Deompressor obtained from CodecPool is already finished()");
344
345
346 }
347 decompressor.reset();
348 }
349 return decompressor;
350 }
351
352 return null;
353 }
354
355 public void returnDecompressor(Decompressor decompressor) {
356 if (decompressor != null) {
357 CodecPool.returnDecompressor(decompressor);
358 if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
359 decompressor.end();
360 }
361 }
362 }
363
364 public String getName() {
365 return compressName;
366 }
367 }
368
369 public static Algorithm getCompressionAlgorithmByName(String compressName) {
370 Algorithm[] algos = Algorithm.class.getEnumConstants();
371
372 for (Algorithm a : algos) {
373 if (a.getName().equals(compressName)) {
374 return a;
375 }
376 }
377
378 throw new IllegalArgumentException(
379 "Unsupported compression algorithm name: " + compressName);
380 }
381
382
383
384
385
386
387
388 public static String[] getSupportedAlgorithms() {
389 Algorithm[] algos = Algorithm.class.getEnumConstants();
390
391 String[] ret = new String[algos.length];
392 int i = 0;
393 for (Algorithm a : algos) {
394 ret[i++] = a.getName();
395 }
396
397 return ret;
398 }
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420 public static void decompress(byte[] dest, int destOffset,
421 InputStream bufferedBoundedStream, int compressedSize,
422 int uncompressedSize, Compression.Algorithm compressAlgo)
423 throws IOException {
424
425 if (dest.length - destOffset < uncompressedSize) {
426 throw new IllegalArgumentException(
427 "Output buffer does not have enough space to hold "
428 + uncompressedSize + " decompressed bytes, available: "
429 + (dest.length - destOffset));
430 }
431
432 Decompressor decompressor = null;
433 try {
434 decompressor = compressAlgo.getDecompressor();
435 InputStream is = compressAlgo.createDecompressionStream(
436 bufferedBoundedStream, decompressor, 0);
437
438 IOUtils.readFully(is, dest, destOffset, uncompressedSize);
439 is.close();
440 } finally {
441 if (decompressor != null) {
442 compressAlgo.returnDecompressor(decompressor);
443 }
444 }
445 }
446
447 }