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