1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
20
21
22 import com.google.protobuf.InvalidProtocolBufferException;
23 import org.apache.hadoop.hbase.testclassification.SmallTests;
24 import org.junit.experimental.categories.Category;
25
26 import java.io.ByteArrayInputStream;
27 import java.io.ByteArrayOutputStream;
28
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.HBaseTestingUtility;
31 import org.apache.hadoop.hbase.KeyValue;
32 import org.apache.hadoop.hbase.client.Put;
33 import org.apache.hadoop.hbase.client.Mutation;
34 import org.apache.hadoop.hbase.client.Result;
35
36 import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
37 import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
38 import org.apache.hadoop.hbase.util.Bytes;
39 import org.apache.hadoop.io.serializer.Deserializer;
40 import org.apache.hadoop.io.serializer.Serializer;
41 import org.junit.BeforeClass;
42 import org.junit.Rule;
43 import org.junit.Test;
44 import org.junit.rules.TestName;
45
46 import static org.junit.Assert.assertTrue;
47 import static org.junit.Assert.fail;
48
49 @Category(SmallTests.class)
50 public class TestSerialization {
51 @Rule public TestName name = new TestName();
52 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
53 private static Configuration conf;
54 private static final byte [] row = Bytes.toBytes("row1");
55 private static final byte [] qualifier = Bytes.toBytes("qualifier1");
56 private static final byte [] family = Bytes.toBytes("family1");
57 private static final byte [] value = new byte[100 * 1024 * 1024];
58
59 @BeforeClass
60 public static void setUpBeforeClass() throws Exception {
61 conf = TEST_UTIL.getConfiguration();
62 conf.setInt("hbase.client.keyvalue.maxsize", Integer.MAX_VALUE);
63 }
64
65 @Test
66 public void testLargeMutation()
67 throws Exception {
68 Put put = new Put(row);
69 put.addColumn(family, qualifier, value);
70
71 MutationSerialization serialization = new MutationSerialization();
72 Serializer<Mutation> serializer = serialization.getSerializer(Mutation.class);
73 Deserializer<Mutation> deserializer = serialization.getDeserializer(Mutation.class);
74 ByteArrayOutputStream os = new ByteArrayOutputStream();
75 ByteArrayInputStream is = null;
76 try {
77 serializer.open(os);
78 serializer.serialize(put);
79 os.flush();
80 is = new ByteArrayInputStream(os.toByteArray());
81 deserializer.open(is);
82 deserializer.deserialize(null);
83 } catch (InvalidProtocolBufferException e) {
84 assertTrue("Got InvalidProtocolBufferException in " + name.getMethodName(),
85 e.getCause() instanceof InvalidProtocolBufferException);
86 } catch (Exception e) {
87 fail("Got an invalid exception: " + e);
88 }
89 }
90
91 @Test
92 public void testLargeResult()
93 throws Exception {
94 Result res = Result.create(new KeyValue[] {new KeyValue(row, family, qualifier, 0L, value)});
95
96 ResultSerialization serialization = new ResultSerialization();
97 Serializer<Result> serializer = serialization.getSerializer(Result.class);
98 Deserializer<Result> deserializer = serialization.getDeserializer(Result.class);
99 ByteArrayOutputStream os = new ByteArrayOutputStream();
100 ByteArrayInputStream is = null;
101 try {
102 serializer.open(os);
103 serializer.serialize(res);
104 os.flush();
105 is = new ByteArrayInputStream(os.toByteArray());
106 deserializer.open(is);
107 deserializer.deserialize(null);
108 } catch (InvalidProtocolBufferException e) {
109 assertTrue("Got InvalidProtocolBufferException in " + name.getMethodName(),
110 e.getCause() instanceof InvalidProtocolBufferException);
111 } catch (Exception e) {
112 fail("Got an invalid exception: " + e);
113 }
114 }
115 }