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