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 java.io.ByteArrayInputStream;
21 import java.io.DataInput;
22 import java.io.IOException;
23 import java.io.InputStream;
24 import java.io.OutputStream;
25 import java.nio.BufferOverflowException;
26 import java.nio.ByteBuffer;
27
28 import org.apache.commons.io.IOUtils;
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configurable;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.CellScanner;
34 import org.apache.hadoop.hbase.DoNotRetryIOException;
35 import org.apache.hadoop.hbase.HBaseIOException;
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.io.HeapSize;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.ClassSize;
44 import org.apache.hadoop.io.compress.CodecPool;
45 import org.apache.hadoop.io.compress.CompressionCodec;
46 import org.apache.hadoop.io.compress.CompressionInputStream;
47 import org.apache.hadoop.io.compress.Compressor;
48 import org.apache.hadoop.io.compress.Decompressor;
49
50 import com.google.common.base.Preconditions;
51 import com.google.protobuf.CodedOutputStream;
52 import com.google.protobuf.Message;
53
54
55
56
57 @InterfaceAudience.Private
58 class IPCUtil {
59 public static final Log LOG = LogFactory.getLog(IPCUtil.class);
60
61
62
63 private final int cellBlockDecompressionMultiplier;
64 private final int cellBlockBuildingInitialBufferSize;
65 private final Configuration conf;
66
67 IPCUtil(final Configuration conf) {
68 super();
69 this.conf = conf;
70 this.cellBlockDecompressionMultiplier =
71 conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3);
72
73
74
75 this.cellBlockBuildingInitialBufferSize =
76 ClassSize.align(conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024));
77 }
78
79
80
81
82 @InterfaceAudience.Private
83 public static class CellScannerButNoCodecException extends HBaseIOException {};
84
85
86
87
88
89
90
91
92
93
94
95
96 @SuppressWarnings("resource")
97 ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
98 final CellScanner cellScanner)
99 throws IOException {
100 return buildCellBlock(codec, compressor, cellScanner, null);
101 }
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118 @SuppressWarnings("resource")
119 public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
120 final CellScanner cellScanner, final BoundedByteBufferPool pool)
121 throws IOException {
122 if (cellScanner == null) return null;
123 if (codec == null) throw new CellScannerButNoCodecException();
124 int bufferSize = this.cellBlockBuildingInitialBufferSize;
125 ByteBufferOutputStream baos = null;
126 if (pool != null) {
127 ByteBuffer bb = pool.getBuffer();
128 bufferSize = bb.capacity();
129 baos = new ByteBufferOutputStream(bb);
130 } else {
131
132 if (cellScanner instanceof HeapSize) {
133 long longSize = ((HeapSize)cellScanner).heapSize();
134
135 if (longSize > Integer.MAX_VALUE) {
136 throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE);
137 }
138 bufferSize = ClassSize.align((int)longSize);
139 }
140 baos = new ByteBufferOutputStream(bufferSize);
141 }
142 OutputStream os = baos;
143 Compressor poolCompressor = null;
144 try {
145 if (compressor != null) {
146 if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
147 poolCompressor = CodecPool.getCompressor(compressor);
148 os = compressor.createOutputStream(os, poolCompressor);
149 }
150 Codec.Encoder encoder = codec.getEncoder(os);
151 int count = 0;
152 while (cellScanner.advance()) {
153 encoder.write(cellScanner.current());
154 count++;
155 }
156 encoder.flush();
157
158
159 if (count == 0) return null;
160 } catch (BufferOverflowException e) {
161 throw new DoNotRetryIOException(e);
162 } finally {
163 os.close();
164 if (poolCompressor != null) CodecPool.returnCompressor(poolCompressor);
165 }
166 if (LOG.isTraceEnabled()) {
167 if (bufferSize < baos.size()) {
168 LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size() +
169 "; up hbase.ipc.cellblock.building.initial.buffersize?");
170 }
171 }
172 return baos.getByteBuffer();
173 }
174
175
176
177
178
179
180
181 CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
182 final byte [] cellBlock)
183 throws IOException {
184 return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock));
185 }
186
187
188
189
190
191
192
193
194 CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
195 final ByteBuffer cellBlock)
196 throws IOException {
197
198
199 InputStream is = null;
200 if (compressor != null) {
201
202 if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
203 Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
204 CompressionInputStream cis =
205 compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor);
206 ByteBufferOutputStream bbos = null;
207 try {
208
209
210 bbos = new ByteBufferOutputStream(cellBlock.remaining() *
211 this.cellBlockDecompressionMultiplier);
212 IOUtils.copy(cis, bbos);
213 bbos.close();
214 ByteBuffer bb = bbos.getByteBuffer();
215 is = new ByteArrayInputStream(bb.array(), 0, bb.limit());
216 } finally {
217 if (is != null) is.close();
218 if (bbos != null) bbos.close();
219
220 CodecPool.returnDecompressor(poolDecompressor);
221 }
222 } else {
223 is = new ByteBufferInputStream(cellBlock);
224 }
225 return codec.getDecoder(is);
226 }
227
228
229
230
231
232
233
234 static ByteBuffer getDelimitedMessageAsByteBuffer(final Message m) throws IOException {
235 if (m == null) return null;
236 int serializedSize = m.getSerializedSize();
237 int vintSize = CodedOutputStream.computeRawVarint32Size(serializedSize);
238 byte [] buffer = new byte[serializedSize + vintSize];
239
240 CodedOutputStream cos = CodedOutputStream.newInstance(buffer);
241
242 cos.writeMessageNoTag(m);
243 cos.flush();
244 cos.checkNoSpaceLeft();
245 return ByteBuffer.wrap(buffer);
246 }
247
248
249
250
251
252
253
254
255
256
257 static int write(final OutputStream dos, final Message header, final Message param,
258 final ByteBuffer cellBlock)
259 throws IOException {
260
261
262
263 int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header, param);
264 if (cellBlock != null) totalSize += cellBlock.remaining();
265 return write(dos, header, param, cellBlock, totalSize);
266 }
267
268 private static int write(final OutputStream dos, final Message header, final Message param,
269 final ByteBuffer cellBlock, final int totalSize)
270 throws IOException {
271
272 dos.write(Bytes.toBytes(totalSize));
273
274 header.writeDelimitedTo(dos);
275 if (param != null) param.writeDelimitedTo(dos);
276 if (cellBlock != null) dos.write(cellBlock.array(), 0, cellBlock.remaining());
277 dos.flush();
278 return totalSize;
279 }
280
281
282
283
284
285
286
287
288
289 static void readChunked(final DataInput in, byte[] dest, int offset, int len)
290 throws IOException {
291 int maxRead = 8192;
292
293 for (; offset < len; offset += maxRead) {
294 in.readFully(dest, offset, Math.min(len - offset, maxRead));
295 }
296 }
297
298
299
300
301
302
303 static int getTotalSizeWhenWrittenDelimited(Message ... messages) {
304 int totalSize = 0;
305 for (Message m: messages) {
306 if (m == null) continue;
307 totalSize += m.getSerializedSize();
308 totalSize += CodedOutputStream.computeRawVarint32Size(m.getSerializedSize());
309 }
310 Preconditions.checkArgument(totalSize < Integer.MAX_VALUE);
311 return totalSize;
312 }
313 }