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