1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.wal;
19
20 import java.io.ByteArrayOutputStream;
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.io.OutputStream;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.Cell;
27 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
28 import org.apache.hadoop.hbase.KeyValue;
29 import org.apache.hadoop.hbase.KeyValueUtil;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.codec.BaseDecoder;
32 import org.apache.hadoop.hbase.codec.BaseEncoder;
33 import org.apache.hadoop.hbase.codec.Codec;
34 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
35 import org.apache.hadoop.hbase.io.util.Dictionary;
36 import org.apache.hadoop.hbase.io.util.StreamUtils;
37 import org.apache.hadoop.hbase.util.ByteStringer;
38 import org.apache.hadoop.hbase.util.Bytes;
39 import org.apache.hadoop.hbase.util.ReflectionUtils;
40 import org.apache.hadoop.io.IOUtils;
41
42 import com.google.protobuf.ByteString;
43
44
45
46
47
48
49
50
51
52 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
53 public class WALCellCodec implements Codec {
54
55 public static final String WAL_CELL_CODEC_CLASS_KEY = "hbase.regionserver.wal.codec";
56
57 protected final CompressionContext compression;
58 protected final ByteStringUncompressor statelessUncompressor = new ByteStringUncompressor() {
59 @Override
60 public byte[] uncompress(ByteString data, Dictionary dict) throws IOException {
61 return WALCellCodec.uncompressByteString(data, dict);
62 }
63 };
64
65
66
67
68 public WALCellCodec() {
69 this.compression = null;
70 }
71
72
73
74
75
76
77
78
79 public WALCellCodec(Configuration conf, CompressionContext compression) {
80 this.compression = compression;
81 }
82
83 public static Class<?> getWALCellCodecClass(Configuration conf) {
84 return conf.getClass(WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class);
85 }
86
87
88
89
90
91
92
93
94
95
96
97
98
99 public static WALCellCodec create(Configuration conf, String cellCodecClsName,
100 CompressionContext compression) throws UnsupportedOperationException {
101 if (cellCodecClsName == null) {
102 cellCodecClsName = getWALCellCodecClass(conf).getName();
103 }
104 return ReflectionUtils.instantiateWithCustomCtor(cellCodecClsName, new Class[]
105 { Configuration.class, CompressionContext.class }, new Object[] { conf, compression });
106 }
107
108
109
110
111
112
113
114
115
116
117
118
119 public static WALCellCodec create(Configuration conf,
120 CompressionContext compression) throws UnsupportedOperationException {
121 String cellCodecClsName = getWALCellCodecClass(conf).getName();
122 return ReflectionUtils.instantiateWithCustomCtor(cellCodecClsName, new Class[]
123 { Configuration.class, CompressionContext.class }, new Object[] { conf, compression });
124 }
125
126 public interface ByteStringCompressor {
127 ByteString compress(byte[] data, Dictionary dict) throws IOException;
128 }
129
130 public interface ByteStringUncompressor {
131 byte[] uncompress(ByteString data, Dictionary dict) throws IOException;
132 }
133
134
135
136
137 static class BaosAndCompressor extends ByteArrayOutputStream implements ByteStringCompressor {
138 public ByteString toByteString() {
139 return ByteString.copyFrom(this.buf, 0, this.count);
140 }
141
142 @Override
143 public ByteString compress(byte[] data, Dictionary dict) throws IOException {
144 writeCompressed(data, dict);
145 ByteString result = ByteString.copyFrom(this.buf, 0, this.count);
146 reset();
147 return result;
148 }
149
150 private void writeCompressed(byte[] data, Dictionary dict) throws IOException {
151 assert dict != null;
152 short dictIdx = dict.findEntry(data, 0, data.length);
153 if (dictIdx == Dictionary.NOT_IN_DICTIONARY) {
154 write(Dictionary.NOT_IN_DICTIONARY);
155 StreamUtils.writeRawVInt32(this, data.length);
156 write(data, 0, data.length);
157 } else {
158 StreamUtils.writeShort(this, dictIdx);
159 }
160 }
161 }
162
163 private static byte[] uncompressByteString(ByteString bs, Dictionary dict) throws IOException {
164 InputStream in = bs.newInput();
165 byte status = (byte)in.read();
166 if (status == Dictionary.NOT_IN_DICTIONARY) {
167 byte[] arr = new byte[StreamUtils.readRawVarint32(in)];
168 int bytesRead = in.read(arr);
169 if (bytesRead != arr.length) {
170 throw new IOException("Cannot read; wanted " + arr.length + ", but got " + bytesRead);
171 }
172 if (dict != null) dict.addEntry(arr, 0, arr.length);
173 return arr;
174 } else {
175
176 short dictIdx = StreamUtils.toShort(status, (byte)in.read());
177 byte[] entry = dict.getEntry(dictIdx);
178 if (entry == null) {
179 throw new IOException("Missing dictionary entry for index " + dictIdx);
180 }
181 return entry;
182 }
183 }
184
185 static class NoneCompressor implements ByteStringCompressor {
186
187 @Override
188 public ByteString compress(byte[] data, Dictionary dict) {
189 return ByteStringer.wrap(data);
190 }
191 }
192
193 static class NoneUncompressor implements ByteStringUncompressor {
194
195 @Override
196 public byte[] uncompress(ByteString data, Dictionary dictIndex) {
197 return data.toByteArray();
198 }
199 }
200
201 static class CompressedKvEncoder extends BaseEncoder {
202 private final CompressionContext compression;
203 public CompressedKvEncoder(OutputStream out, CompressionContext compression) {
204 super(out);
205 this.compression = compression;
206 }
207
208 @Override
209 public void write(Cell cell) throws IOException {
210
211 StreamUtils.writeRawVInt32(out, KeyValueUtil.keyLength(cell));
212 StreamUtils.writeRawVInt32(out, cell.getValueLength());
213
214 int tagsLength = cell.getTagsLength();
215 StreamUtils.writeRawVInt32(out, tagsLength);
216
217
218
219 write(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), compression.rowDict);
220 write(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
221 compression.familyDict);
222 write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
223 compression.qualifierDict);
224
225
226 StreamUtils.writeLong(out, cell.getTimestamp());
227 out.write(cell.getTypeByte());
228 out.write(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
229 if (tagsLength > 0) {
230 if (compression.tagCompressionContext != null) {
231
232 compression.tagCompressionContext.compressTags(out, cell.getTagsArray(),
233 cell.getTagsOffset(), tagsLength);
234 } else {
235
236
237 out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
238 }
239 }
240 }
241
242 private void write(byte[] data, int offset, int length, Dictionary dict) throws IOException {
243 short dictIdx = Dictionary.NOT_IN_DICTIONARY;
244 if (dict != null) {
245 dictIdx = dict.findEntry(data, offset, length);
246 }
247 if (dictIdx == Dictionary.NOT_IN_DICTIONARY) {
248 out.write(Dictionary.NOT_IN_DICTIONARY);
249 StreamUtils.writeRawVInt32(out, length);
250 out.write(data, offset, length);
251 } else {
252 StreamUtils.writeShort(out, dictIdx);
253 }
254 }
255 }
256
257 static class CompressedKvDecoder extends BaseDecoder {
258 private final CompressionContext compression;
259 public CompressedKvDecoder(InputStream in, CompressionContext compression) {
260 super(in);
261 this.compression = compression;
262 }
263
264 @Override
265 protected Cell parseCell() throws IOException {
266 int keylength = StreamUtils.readRawVarint32(in);
267 int vlength = StreamUtils.readRawVarint32(in);
268
269 int tagsLength = StreamUtils.readRawVarint32(in);
270 int length = 0;
271 if(tagsLength == 0) {
272 length = KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + keylength + vlength;
273 } else {
274 length = KeyValue.KEYVALUE_WITH_TAGS_INFRASTRUCTURE_SIZE + keylength + vlength + tagsLength;
275 }
276
277 byte[] backingArray = new byte[length];
278 int pos = 0;
279 pos = Bytes.putInt(backingArray, pos, keylength);
280 pos = Bytes.putInt(backingArray, pos, vlength);
281
282
283 int elemLen = readIntoArray(backingArray, pos + Bytes.SIZEOF_SHORT, compression.rowDict);
284 checkLength(elemLen, Short.MAX_VALUE);
285 pos = Bytes.putShort(backingArray, pos, (short)elemLen);
286 pos += elemLen;
287
288
289 elemLen = readIntoArray(backingArray, pos + Bytes.SIZEOF_BYTE, compression.familyDict);
290 checkLength(elemLen, Byte.MAX_VALUE);
291 pos = Bytes.putByte(backingArray, pos, (byte)elemLen);
292 pos += elemLen;
293
294
295 elemLen = readIntoArray(backingArray, pos, compression.qualifierDict);
296 pos += elemLen;
297
298
299 int tsTypeValLen = length - pos;
300 if (tagsLength > 0) {
301 tsTypeValLen = tsTypeValLen - tagsLength - KeyValue.TAGS_LENGTH_SIZE;
302 }
303 IOUtils.readFully(in, backingArray, pos, tsTypeValLen);
304 pos += tsTypeValLen;
305
306
307 if (tagsLength > 0) {
308 pos = Bytes.putAsShort(backingArray, pos, tagsLength);
309 if (compression.tagCompressionContext != null) {
310 compression.tagCompressionContext.uncompressTags(in, backingArray, pos, tagsLength);
311 } else {
312 IOUtils.readFully(in, backingArray, pos, tagsLength);
313 }
314 }
315 return new KeyValue(backingArray, 0, length);
316 }
317
318 private int readIntoArray(byte[] to, int offset, Dictionary dict) throws IOException {
319 byte status = (byte)in.read();
320 if (status == Dictionary.NOT_IN_DICTIONARY) {
321
322
323 int length = StreamUtils.readRawVarint32(in);
324 IOUtils.readFully(in, to, offset, length);
325 dict.addEntry(to, offset, length);
326 return length;
327 } else {
328
329 short dictIdx = StreamUtils.toShort(status, (byte)in.read());
330 byte[] entry = dict.getEntry(dictIdx);
331 if (entry == null) {
332 throw new IOException("Missing dictionary entry for index " + dictIdx);
333 }
334
335 Bytes.putBytes(to, offset, entry, 0, entry.length);
336 return entry.length;
337 }
338 }
339
340 private static void checkLength(int len, int max) throws IOException {
341 if (len < 0 || len > max) {
342 throw new IOException("Invalid length for compresesed portion of keyvalue: " + len);
343 }
344 }
345 }
346
347 public static class EnsureKvEncoder extends BaseEncoder {
348 public EnsureKvEncoder(OutputStream out) {
349 super(out);
350 }
351 @Override
352 public void write(Cell cell) throws IOException {
353 checkFlushed();
354
355 KeyValueUtil.oswrite(cell, this.out, true);
356 }
357 }
358
359 @Override
360 public Decoder getDecoder(InputStream is) {
361 return (compression == null)
362 ? new KeyValueCodecWithTags.KeyValueDecoder(is) : new CompressedKvDecoder(is, compression);
363 }
364
365 @Override
366 public Encoder getEncoder(OutputStream os) {
367 return (compression == null)
368 ? new EnsureKvEncoder(os) : new CompressedKvEncoder(os, compression);
369 }
370
371 public ByteStringCompressor getByteStringCompressor() {
372
373 return new BaosAndCompressor();
374 }
375
376 public ByteStringUncompressor getByteStringUncompressor() {
377
378 return this.statelessUncompressor;
379 }
380
381 public static ByteStringCompressor getNoneCompressor() {
382 return new NoneCompressor();
383 }
384
385 public static ByteStringUncompressor getNoneUncompressor() {
386 return new NoneUncompressor();
387 }
388 }