1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.protobuf;
20
21 import static org.junit.Assert.assertEquals;
22
23 import java.io.IOException;
24
25 import org.apache.hadoop.hbase.util.ByteStringer;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.testclassification.SmallTests;
28 import org.apache.hadoop.hbase.client.Append;
29 import org.apache.hadoop.hbase.client.Delete;
30 import org.apache.hadoop.hbase.client.Get;
31 import org.apache.hadoop.hbase.client.Increment;
32 import org.apache.hadoop.hbase.client.Put;
33 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
34 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
35 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
36 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
37 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
38 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
39 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
40 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
41 import org.apache.hadoop.hbase.util.Bytes;
42 import org.junit.Test;
43 import org.junit.experimental.categories.Category;
44
45 import com.google.protobuf.ByteString;
46
47
48
49
50 @Category(SmallTests.class)
51 public class TestProtobufUtil {
52 @Test
53 public void testException() throws IOException {
54 NameBytesPair.Builder builder = NameBytesPair.newBuilder();
55 final String omg = "OMG!!!";
56 builder.setName("java.io.IOException");
57 builder.setValue(ByteStringer.wrap(Bytes.toBytes(omg)));
58 Throwable t = ProtobufUtil.toException(builder.build());
59 assertEquals(omg, t.getMessage());
60 builder.clear();
61 builder.setName("org.apache.hadoop.ipc.RemoteException");
62 builder.setValue(ByteStringer.wrap(Bytes.toBytes(omg)));
63 t = ProtobufUtil.toException(builder.build());
64 assertEquals(omg, t.getMessage());
65 }
66
67
68
69
70
71
72 @Test
73 public void testGet() throws IOException {
74 ClientProtos.Get.Builder getBuilder = ClientProtos.Get.newBuilder();
75 getBuilder.setRow(ByteString.copyFromUtf8("row"));
76 Column.Builder columnBuilder = Column.newBuilder();
77 columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
78 columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
79 columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
80 getBuilder.addColumn(columnBuilder.build());
81
82 columnBuilder.clear();
83 columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
84 getBuilder.addColumn(columnBuilder.build());
85
86 ClientProtos.Get proto = getBuilder.build();
87
88 assertEquals(1, proto.getMaxVersions());
89 assertEquals(true, proto.getCacheBlocks());
90
91
92 getBuilder = ClientProtos.Get.newBuilder(proto);
93 getBuilder.setMaxVersions(1);
94 getBuilder.setCacheBlocks(true);
95
96 Get get = ProtobufUtil.toGet(proto);
97 assertEquals(getBuilder.build(), ProtobufUtil.toGet(get));
98 }
99
100
101
102
103
104
105 @Test
106 public void testAppend() throws IOException {
107 long timeStamp = 111111;
108 MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
109 mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
110 mutateBuilder.setMutateType(MutationType.APPEND);
111 mutateBuilder.setTimestamp(timeStamp);
112 ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
113 valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
114 QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
115 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
116 qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
117 qualifierBuilder.setTimestamp(timeStamp);
118 valueBuilder.addQualifierValue(qualifierBuilder.build());
119 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
120 qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
121 valueBuilder.addQualifierValue(qualifierBuilder.build());
122 qualifierBuilder.setTimestamp(timeStamp);
123 mutateBuilder.addColumnValue(valueBuilder.build());
124
125 MutationProto proto = mutateBuilder.build();
126
127 assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
128
129
130 mutateBuilder = MutationProto.newBuilder(proto);
131 mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
132
133 Append append = ProtobufUtil.toAppend(proto, null);
134
135
136
137 mutateBuilder.setTimestamp(append.getTimeStamp());
138 assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
139 }
140
141
142
143
144
145
146 @Test
147 public void testDelete() throws IOException {
148 MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
149 mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
150 mutateBuilder.setMutateType(MutationType.DELETE);
151 mutateBuilder.setTimestamp(111111);
152 ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
153 valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
154 QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
155 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
156 qualifierBuilder.setDeleteType(DeleteType.DELETE_ONE_VERSION);
157 qualifierBuilder.setTimestamp(111222);
158 valueBuilder.addQualifierValue(qualifierBuilder.build());
159 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
160 qualifierBuilder.setDeleteType(DeleteType.DELETE_MULTIPLE_VERSIONS);
161 qualifierBuilder.setTimestamp(111333);
162 valueBuilder.addQualifierValue(qualifierBuilder.build());
163 mutateBuilder.addColumnValue(valueBuilder.build());
164
165 MutationProto proto = mutateBuilder.build();
166
167 assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
168
169
170 mutateBuilder = MutationProto.newBuilder(proto);
171 mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
172
173 Delete delete = ProtobufUtil.toDelete(proto);
174
175
176
177 for (ColumnValue.Builder column:
178 mutateBuilder.getColumnValueBuilderList()) {
179 for (QualifierValue.Builder qualifier:
180 column.getQualifierValueBuilderList()) {
181 qualifier.setValue(ByteString.EMPTY);
182 }
183 }
184 assertEquals(mutateBuilder.build(),
185 ProtobufUtil.toMutation(MutationType.DELETE, delete));
186 }
187
188
189
190
191
192
193 @Test
194 public void testIncrement() throws IOException {
195 MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
196 mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
197 mutateBuilder.setMutateType(MutationType.INCREMENT);
198 ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
199 valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
200 QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
201 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
202 qualifierBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(11L)));
203 valueBuilder.addQualifierValue(qualifierBuilder.build());
204 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
205 qualifierBuilder.setValue(ByteStringer.wrap(Bytes.toBytes(22L)));
206 valueBuilder.addQualifierValue(qualifierBuilder.build());
207 mutateBuilder.addColumnValue(valueBuilder.build());
208
209 MutationProto proto = mutateBuilder.build();
210
211 assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
212
213
214 mutateBuilder = MutationProto.newBuilder(proto);
215 mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
216
217 Increment increment = ProtobufUtil.toIncrement(proto, null);
218 assertEquals(mutateBuilder.build(),
219 ProtobufUtil.toMutation(increment, MutationProto.newBuilder(), HConstants.NO_NONCE));
220 }
221
222
223
224
225
226
227 @Test
228 public void testPut() throws IOException {
229 MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
230 mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
231 mutateBuilder.setMutateType(MutationType.PUT);
232 mutateBuilder.setTimestamp(111111);
233 ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
234 valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
235 QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
236 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
237 qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
238 valueBuilder.addQualifierValue(qualifierBuilder.build());
239 qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
240 qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
241 qualifierBuilder.setTimestamp(222222);
242 valueBuilder.addQualifierValue(qualifierBuilder.build());
243 mutateBuilder.addColumnValue(valueBuilder.build());
244
245 MutationProto proto = mutateBuilder.build();
246
247 assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
248
249
250 mutateBuilder = MutationProto.newBuilder(proto);
251 mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
252
253 Put put = ProtobufUtil.toPut(proto);
254
255
256
257
258 long timestamp = put.getTimeStamp();
259 for (ColumnValue.Builder column:
260 mutateBuilder.getColumnValueBuilderList()) {
261 for (QualifierValue.Builder qualifier:
262 column.getQualifierValueBuilderList()) {
263 if (!qualifier.hasTimestamp()) {
264 qualifier.setTimestamp(timestamp);
265 }
266 }
267 }
268 assertEquals(mutateBuilder.build(),
269 ProtobufUtil.toMutation(MutationType.PUT, put));
270 }
271
272
273
274
275
276
277 @Test
278 public void testScan() throws IOException {
279 ClientProtos.Scan.Builder scanBuilder = ClientProtos.Scan.newBuilder();
280 scanBuilder.setStartRow(ByteString.copyFromUtf8("row1"));
281 scanBuilder.setStopRow(ByteString.copyFromUtf8("row2"));
282 Column.Builder columnBuilder = Column.newBuilder();
283 columnBuilder.setFamily(ByteString.copyFromUtf8("f1"));
284 columnBuilder.addQualifier(ByteString.copyFromUtf8("c1"));
285 columnBuilder.addQualifier(ByteString.copyFromUtf8("c2"));
286 scanBuilder.addColumn(columnBuilder.build());
287
288 columnBuilder.clear();
289 columnBuilder.setFamily(ByteString.copyFromUtf8("f2"));
290 scanBuilder.addColumn(columnBuilder.build());
291
292 ClientProtos.Scan proto = scanBuilder.build();
293
294
295 assertEquals(1, proto.getMaxVersions());
296 assertEquals(true, proto.getCacheBlocks());
297
298
299
300 scanBuilder = ClientProtos.Scan.newBuilder(proto);
301 scanBuilder.setMaxVersions(2);
302 scanBuilder.setCacheBlocks(false);
303 scanBuilder.setCaching(1024);
304 ClientProtos.Scan expectedProto = scanBuilder.build();
305
306 ClientProtos.Scan actualProto = ProtobufUtil.toScan(
307 ProtobufUtil.toScan(expectedProto));
308 assertEquals(expectedProto, actualProto);
309 }
310 }