1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.ipc;
19
20 import io.netty.buffer.ByteBuf;
21 import io.netty.buffer.ByteBufAllocator;
22 import io.netty.buffer.ByteBufOutputStream;
23
24 import java.io.IOException;
25 import java.io.OutputStream;
26 import java.nio.BufferOverflowException;
27 import java.nio.ByteBuffer;
28
29 import org.apache.commons.io.IOUtils;
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configurable;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.hbase.CellScanner;
35 import org.apache.hadoop.hbase.DoNotRetryIOException;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.codec.Codec;
38 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
39 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
40 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
41 import org.apache.hadoop.hbase.util.ClassSize;
42 import org.apache.hadoop.io.compress.CodecPool;
43 import org.apache.hadoop.io.compress.CompressionCodec;
44 import org.apache.hadoop.io.compress.CompressionInputStream;
45 import org.apache.hadoop.io.compress.Compressor;
46 import org.apache.hadoop.io.compress.Decompressor;
47
48
49
50
51 @InterfaceAudience.Private
52 class CellBlockBuilder {
53
54
55 static final Log LOG = LogFactory.getLog(CellBlockBuilder.class);
56
57 private final Configuration conf;
58
59
60
61
62 private final int cellBlockDecompressionMultiplier;
63
64 private final int cellBlockBuildingInitialBufferSize;
65
66 public CellBlockBuilder(Configuration conf) {
67 this.conf = conf;
68 this.cellBlockDecompressionMultiplier =
69 conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3);
70
71
72
73 this.cellBlockBuildingInitialBufferSize =
74 ClassSize.align(conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024));
75 }
76
77 private interface OutputStreamSupplier {
78
79 OutputStream get(int expectedSize);
80
81 int size();
82 }
83
84 private static final class ByteBufferOutputStreamSupplier implements OutputStreamSupplier {
85
86 private ByteBufferOutputStream baos;
87
88 @Override
89 public OutputStream get(int expectedSize) {
90 baos = new ByteBufferOutputStream(expectedSize);
91 return baos;
92 }
93
94 @Override
95 public int size() {
96 return baos.size();
97 }
98 }
99
100
101
102
103
104
105
106
107
108
109
110
111 public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
112 final CellScanner cellScanner) throws IOException {
113 ByteBufferOutputStreamSupplier supplier = new ByteBufferOutputStreamSupplier();
114 if (buildCellBlock(codec, compressor, cellScanner, supplier)) {
115 ByteBuffer bb = supplier.baos.getByteBuffer();
116
117
118 return bb.hasRemaining() ? bb : null;
119 } else {
120 return null;
121 }
122 }
123
124 private static final class ByteBufOutputStreamSupplier implements OutputStreamSupplier {
125
126 private final ByteBufAllocator alloc;
127
128 private ByteBuf buf;
129
130 public ByteBufOutputStreamSupplier(ByteBufAllocator alloc) {
131 this.alloc = alloc;
132 }
133
134 @Override
135 public OutputStream get(int expectedSize) {
136 buf = alloc.buffer(expectedSize);
137 return new ByteBufOutputStream(buf);
138 }
139
140 @Override
141 public int size() {
142 return buf.writerIndex();
143 }
144 }
145
146 public ByteBuf buildCellBlock(Codec codec, CompressionCodec compressor, CellScanner cellScanner,
147 ByteBufAllocator alloc) throws IOException {
148 ByteBufOutputStreamSupplier supplier = new ByteBufOutputStreamSupplier(alloc);
149 if (buildCellBlock(codec, compressor, cellScanner, supplier)) {
150 return supplier.buf;
151 } else {
152 return null;
153 }
154 }
155
156 private boolean buildCellBlock(final Codec codec, final CompressionCodec compressor,
157 final CellScanner cellScanner, OutputStreamSupplier supplier) throws IOException {
158 if (cellScanner == null) {
159 return false;
160 }
161 if (codec == null) {
162 throw new CellScannerButNoCodecException();
163 }
164 int bufferSize = cellBlockBuildingInitialBufferSize;
165 encodeCellsTo(supplier.get(bufferSize), cellScanner, codec, compressor);
166 if (LOG.isTraceEnabled() && bufferSize < supplier.size()) {
167 LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + supplier.size() +
168 "; up hbase.ipc.cellblock.building.initial.buffersize?");
169 }
170 return true;
171 }
172
173 private void encodeCellsTo(OutputStream os, CellScanner cellScanner, Codec codec,
174 CompressionCodec compressor) throws IOException {
175 Compressor poolCompressor = null;
176 try {
177 if (compressor != null) {
178 if (compressor instanceof Configurable) {
179 ((Configurable) compressor).setConf(this.conf);
180 }
181 poolCompressor = CodecPool.getCompressor(compressor);
182 os = compressor.createOutputStream(os, poolCompressor);
183 }
184 Codec.Encoder encoder = codec.getEncoder(os);
185 while (cellScanner.advance()) {
186 encoder.write(cellScanner.current());
187 }
188 encoder.flush();
189 } catch (BufferOverflowException | IndexOutOfBoundsException e) {
190 throw new DoNotRetryIOException(e);
191 } finally {
192 os.close();
193 if (poolCompressor != null) {
194 CodecPool.returnCompressor(poolCompressor);
195 }
196 }
197 }
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214 public ByteBuffer buildCellBlock(Codec codec, CompressionCodec compressor,
215 CellScanner cellScanner, BoundedByteBufferPool pool) throws IOException {
216 if (cellScanner == null) {
217 return null;
218 }
219 if (codec == null) {
220 throw new CellScannerButNoCodecException();
221 }
222 ByteBufferOutputStream bbos;
223 ByteBuffer bb = null;
224 if (pool != null) {
225 bb = pool.getBuffer();
226 bbos = new ByteBufferOutputStream(bb);
227 } else {
228 bbos = new ByteBufferOutputStream(cellBlockBuildingInitialBufferSize);
229 }
230 encodeCellsTo(bbos, cellScanner, codec, compressor);
231 if (bbos.size() == 0) {
232 if (pool != null) {
233 pool.putBuffer(bb);
234 }
235 return null;
236 }
237 return bbos.getByteBuffer();
238 }
239
240
241
242
243
244
245
246 public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
247 final byte[] cellBlock) throws IOException {
248 return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock));
249 }
250
251
252
253
254
255
256
257
258 public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
259 ByteBuffer cellBlock) throws IOException {
260 if (compressor != null) {
261 cellBlock = decompress(compressor, cellBlock);
262 }
263
264
265
266
267 return codec.getDecoder(new ByteBufferInputStream(cellBlock));
268 }
269
270 private ByteBuffer decompress(CompressionCodec compressor, ByteBuffer cellBlock)
271 throws IOException {
272
273 if (compressor instanceof Configurable) {
274 ((Configurable) compressor).setConf(this.conf);
275 }
276 Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
277 CompressionInputStream cis =
278 compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor);
279 ByteBufferOutputStream bbos;
280 try {
281
282
283 bbos =
284 new ByteBufferOutputStream(cellBlock.remaining() * this.cellBlockDecompressionMultiplier);
285 IOUtils.copy(cis, bbos);
286 bbos.close();
287 cellBlock = bbos.getByteBuffer();
288 } finally {
289 CodecPool.returnDecompressor(poolDecompressor);
290 }
291 return cellBlock;
292 }
293 }