1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertNotNull;
23
24 import org.apache.hadoop.hbase.Cell;
25 import org.apache.hadoop.hbase.CellUtil;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.testclassification.SmallTests;
29 import org.junit.Assert;
30 import org.junit.Test;
31
32 import java.io.IOException;
33 import java.nio.ByteBuffer;
34 import java.util.Arrays;
35 import java.util.HashMap;
36 import java.util.List;
37 import java.util.Map;
38
39 import org.apache.hadoop.hbase.filter.BinaryComparator;
40 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
41 import org.apache.hadoop.hbase.filter.ColumnPaginationFilter;
42 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
43 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
44 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
45 import org.apache.hadoop.hbase.filter.DependentColumnFilter;
46 import org.apache.hadoop.hbase.filter.FamilyFilter;
47 import org.apache.hadoop.hbase.filter.Filter;
48 import org.apache.hadoop.hbase.filter.FilterList;
49 import org.apache.hadoop.hbase.filter.FilterList.Operator;
50 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
51 import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
52 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
53 import org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter;
54 import org.apache.hadoop.hbase.filter.PageFilter;
55 import org.apache.hadoop.hbase.filter.PrefixFilter;
56 import org.apache.hadoop.hbase.filter.QualifierFilter;
57 import org.apache.hadoop.hbase.filter.RowFilter;
58 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
59 import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
60 import org.apache.hadoop.hbase.filter.SkipFilter;
61 import org.apache.hadoop.hbase.filter.TimestampsFilter;
62 import org.apache.hadoop.hbase.filter.ValueFilter;
63 import org.apache.hadoop.hbase.filter.WhileMatchFilter;
64 import org.apache.hadoop.hbase.util.Bytes;
65
66 import org.codehaus.jackson.map.ObjectMapper;
67 import org.junit.experimental.categories.Category;
68
69
70
71
72
73 @Category(SmallTests.class)
74 public class TestOperation {
75 private static byte [] ROW = Bytes.toBytes("testRow");
76 private static byte [] FAMILY = Bytes.toBytes("testFamily");
77 private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
78 private static byte [] VALUE = Bytes.toBytes("testValue");
79
80 private static ObjectMapper mapper = new ObjectMapper();
81
82 private static List<Long> TS_LIST = Arrays.asList(2L, 3L, 5L);
83 private static TimestampsFilter TS_FILTER = new TimestampsFilter(TS_LIST);
84 private static String STR_TS_FILTER =
85 TS_FILTER.getClass().getSimpleName() + " (3/3): [2, 3, 5]";
86
87 private static List<Long> L_TS_LIST =
88 Arrays.asList(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L);
89 private static TimestampsFilter L_TS_FILTER =
90 new TimestampsFilter(L_TS_LIST);
91 private static String STR_L_TS_FILTER =
92 L_TS_FILTER.getClass().getSimpleName() + " (5/11): [0, 1, 2, 3, 4]";
93
94 private static String COL_NAME_1 = "col1";
95 private static ColumnPrefixFilter COL_PRE_FILTER =
96 new ColumnPrefixFilter(COL_NAME_1.getBytes());
97 private static String STR_COL_PRE_FILTER =
98 COL_PRE_FILTER.getClass().getSimpleName() + " " + COL_NAME_1;
99
100 private static String COL_NAME_2 = "col2";
101 private static ColumnRangeFilter CR_FILTER = new ColumnRangeFilter(
102 COL_NAME_1.getBytes(), true, COL_NAME_2.getBytes(), false);
103 private static String STR_CR_FILTER = CR_FILTER.getClass().getSimpleName()
104 + " [" + COL_NAME_1 + ", " + COL_NAME_2 + ")";
105
106 private static int COL_COUNT = 9;
107 private static ColumnCountGetFilter CCG_FILTER =
108 new ColumnCountGetFilter(COL_COUNT);
109 private static String STR_CCG_FILTER =
110 CCG_FILTER.getClass().getSimpleName() + " " + COL_COUNT;
111
112 private static int LIMIT = 3;
113 private static int OFFSET = 4;
114 private static ColumnPaginationFilter CP_FILTER =
115 new ColumnPaginationFilter(LIMIT, OFFSET);
116 private static String STR_CP_FILTER = CP_FILTER.getClass().getSimpleName()
117 + " (" + LIMIT + ", " + OFFSET + ")";
118
119 private static String STOP_ROW_KEY = "stop";
120 private static InclusiveStopFilter IS_FILTER =
121 new InclusiveStopFilter(STOP_ROW_KEY.getBytes());
122 private static String STR_IS_FILTER =
123 IS_FILTER.getClass().getSimpleName() + " " + STOP_ROW_KEY;
124
125 private static String PREFIX = "prefix";
126 private static PrefixFilter PREFIX_FILTER =
127 new PrefixFilter(PREFIX.getBytes());
128 private static String STR_PREFIX_FILTER = "PrefixFilter " + PREFIX;
129
130 private static byte[][] PREFIXES = {
131 "0".getBytes(), "1".getBytes(), "2".getBytes()};
132 private static MultipleColumnPrefixFilter MCP_FILTER =
133 new MultipleColumnPrefixFilter(PREFIXES);
134 private static String STR_MCP_FILTER =
135 MCP_FILTER.getClass().getSimpleName() + " (3/3): [0, 1, 2]";
136
137 private static byte[][] L_PREFIXES = {
138 "0".getBytes(), "1".getBytes(), "2".getBytes(), "3".getBytes(),
139 "4".getBytes(), "5".getBytes(), "6".getBytes(), "7".getBytes()};
140 private static MultipleColumnPrefixFilter L_MCP_FILTER =
141 new MultipleColumnPrefixFilter(L_PREFIXES);
142 private static String STR_L_MCP_FILTER =
143 L_MCP_FILTER.getClass().getSimpleName() + " (5/8): [0, 1, 2, 3, 4]";
144
145 private static int PAGE_SIZE = 9;
146 private static PageFilter PAGE_FILTER = new PageFilter(PAGE_SIZE);
147 private static String STR_PAGE_FILTER =
148 PAGE_FILTER.getClass().getSimpleName() + " " + PAGE_SIZE;
149
150 private static SkipFilter SKIP_FILTER = new SkipFilter(L_TS_FILTER);
151 private static String STR_SKIP_FILTER =
152 SKIP_FILTER.getClass().getSimpleName() + " " + STR_L_TS_FILTER;
153
154 private static WhileMatchFilter WHILE_FILTER =
155 new WhileMatchFilter(L_TS_FILTER);
156 private static String STR_WHILE_FILTER =
157 WHILE_FILTER.getClass().getSimpleName() + " " + STR_L_TS_FILTER;
158
159 private static KeyOnlyFilter KEY_ONLY_FILTER = new KeyOnlyFilter();
160 private static String STR_KEY_ONLY_FILTER =
161 KEY_ONLY_FILTER.getClass().getSimpleName();
162
163 private static FirstKeyOnlyFilter FIRST_KEY_ONLY_FILTER =
164 new FirstKeyOnlyFilter();
165 private static String STR_FIRST_KEY_ONLY_FILTER =
166 FIRST_KEY_ONLY_FILTER.getClass().getSimpleName();
167
168 private static CompareOp CMP_OP = CompareOp.EQUAL;
169 private static byte[] CMP_VALUE = "value".getBytes();
170 private static BinaryComparator BC = new BinaryComparator(CMP_VALUE);
171 private static DependentColumnFilter DC_FILTER =
172 new DependentColumnFilter(FAMILY, QUALIFIER, true, CMP_OP, BC);
173 private static String STR_DC_FILTER = String.format(
174 "%s (%s, %s, %s, %s, %s)", DC_FILTER.getClass().getSimpleName(),
175 Bytes.toStringBinary(FAMILY), Bytes.toStringBinary(QUALIFIER), true,
176 CMP_OP.name(), Bytes.toStringBinary(BC.getValue()));
177
178 private static FamilyFilter FAMILY_FILTER = new FamilyFilter(CMP_OP, BC);
179 private static String STR_FAMILY_FILTER =
180 FAMILY_FILTER.getClass().getSimpleName() + " (EQUAL, value)";
181
182 private static QualifierFilter QUALIFIER_FILTER =
183 new QualifierFilter(CMP_OP, BC);
184 private static String STR_QUALIFIER_FILTER =
185 QUALIFIER_FILTER.getClass().getSimpleName() + " (EQUAL, value)";
186
187 private static RowFilter ROW_FILTER = new RowFilter(CMP_OP, BC);
188 private static String STR_ROW_FILTER =
189 ROW_FILTER.getClass().getSimpleName() + " (EQUAL, value)";
190
191 private static ValueFilter VALUE_FILTER = new ValueFilter(CMP_OP, BC);
192 private static String STR_VALUE_FILTER =
193 VALUE_FILTER.getClass().getSimpleName() + " (EQUAL, value)";
194
195 private static SingleColumnValueFilter SCV_FILTER =
196 new SingleColumnValueFilter(FAMILY, QUALIFIER, CMP_OP, CMP_VALUE);
197 private static String STR_SCV_FILTER = String.format("%s (%s, %s, %s, %s)",
198 SCV_FILTER.getClass().getSimpleName(), Bytes.toStringBinary(FAMILY),
199 Bytes.toStringBinary(QUALIFIER), CMP_OP.name(),
200 Bytes.toStringBinary(CMP_VALUE));
201
202 private static SingleColumnValueExcludeFilter SCVE_FILTER =
203 new SingleColumnValueExcludeFilter(FAMILY, QUALIFIER, CMP_OP, CMP_VALUE);
204 private static String STR_SCVE_FILTER = String.format("%s (%s, %s, %s, %s)",
205 SCVE_FILTER.getClass().getSimpleName(), Bytes.toStringBinary(FAMILY),
206 Bytes.toStringBinary(QUALIFIER), CMP_OP.name(),
207 Bytes.toStringBinary(CMP_VALUE));
208
209 private static FilterList AND_FILTER_LIST = new FilterList(
210 Operator.MUST_PASS_ALL, Arrays.asList((Filter) TS_FILTER, L_TS_FILTER,
211 CR_FILTER));
212 private static String STR_AND_FILTER_LIST = String.format(
213 "%s AND (3/3): [%s, %s, %s]", AND_FILTER_LIST.getClass().getSimpleName(),
214 STR_TS_FILTER, STR_L_TS_FILTER, STR_CR_FILTER);
215
216 private static FilterList OR_FILTER_LIST = new FilterList(
217 Operator.MUST_PASS_ONE, Arrays.asList((Filter) TS_FILTER, L_TS_FILTER,
218 CR_FILTER));
219 private static String STR_OR_FILTER_LIST = String.format(
220 "%s OR (3/3): [%s, %s, %s]", AND_FILTER_LIST.getClass().getSimpleName(),
221 STR_TS_FILTER, STR_L_TS_FILTER, STR_CR_FILTER);
222
223 private static FilterList L_FILTER_LIST = new FilterList(
224 Arrays.asList((Filter) TS_FILTER, L_TS_FILTER, CR_FILTER, COL_PRE_FILTER,
225 CCG_FILTER, CP_FILTER, PREFIX_FILTER, PAGE_FILTER));
226 private static String STR_L_FILTER_LIST = String.format(
227 "%s AND (5/8): [%s, %s, %s, %s, %s, %s]",
228 L_FILTER_LIST.getClass().getSimpleName(), STR_TS_FILTER, STR_L_TS_FILTER,
229 STR_CR_FILTER, STR_COL_PRE_FILTER, STR_CCG_FILTER, STR_CP_FILTER);
230
231 private static Filter[] FILTERS = {
232 TS_FILTER,
233 L_TS_FILTER,
234 COL_PRE_FILTER,
235 CP_FILTER,
236 CR_FILTER,
237 CCG_FILTER,
238 IS_FILTER,
239 PREFIX_FILTER,
240 PAGE_FILTER,
241 SKIP_FILTER,
242 WHILE_FILTER,
243 KEY_ONLY_FILTER,
244 FIRST_KEY_ONLY_FILTER,
245 MCP_FILTER,
246 L_MCP_FILTER,
247 DC_FILTER,
248 FAMILY_FILTER,
249 QUALIFIER_FILTER,
250 ROW_FILTER,
251 VALUE_FILTER,
252 SCV_FILTER,
253 SCVE_FILTER,
254 AND_FILTER_LIST,
255 OR_FILTER_LIST,
256 L_FILTER_LIST,
257 };
258
259 private static String[] FILTERS_INFO = {
260 STR_TS_FILTER,
261 STR_L_TS_FILTER,
262 STR_COL_PRE_FILTER,
263 STR_CP_FILTER,
264 STR_CR_FILTER,
265 STR_CCG_FILTER,
266 STR_IS_FILTER,
267 STR_PREFIX_FILTER,
268 STR_PAGE_FILTER,
269 STR_SKIP_FILTER,
270 STR_WHILE_FILTER,
271 STR_KEY_ONLY_FILTER,
272 STR_FIRST_KEY_ONLY_FILTER,
273 STR_MCP_FILTER,
274 STR_L_MCP_FILTER,
275 STR_DC_FILTER,
276 STR_FAMILY_FILTER,
277 STR_QUALIFIER_FILTER,
278 STR_ROW_FILTER,
279 STR_VALUE_FILTER,
280 STR_SCV_FILTER,
281 STR_SCVE_FILTER,
282 STR_AND_FILTER_LIST,
283 STR_OR_FILTER_LIST,
284 STR_L_FILTER_LIST,
285 };
286
287 static {
288 assertEquals("The sizes of static arrays do not match: "
289 + "[FILTERS: %d <=> FILTERS_INFO: %d]",
290 FILTERS.length, FILTERS_INFO.length);
291 }
292
293
294
295
296
297
298 @Test
299 public void testOperationJSON()
300 throws IOException {
301
302 Scan scan = new Scan(ROW);
303 scan.addColumn(FAMILY, QUALIFIER);
304
305 String json = scan.toJSON();
306 Map<String, Object> parsedJSON = mapper.readValue(json, HashMap.class);
307
308 assertEquals("startRow incorrect in Scan.toJSON()",
309 Bytes.toStringBinary(ROW), parsedJSON.get("startRow"));
310
311 List familyInfo = (List) ((Map) parsedJSON.get("families")).get(
312 Bytes.toStringBinary(FAMILY));
313 assertNotNull("Family absent in Scan.toJSON()", familyInfo);
314 assertEquals("Qualifier absent in Scan.toJSON()", 1, familyInfo.size());
315 assertEquals("Qualifier incorrect in Scan.toJSON()",
316 Bytes.toStringBinary(QUALIFIER),
317 familyInfo.get(0));
318
319
320 Get get = new Get(ROW);
321 get.addColumn(FAMILY, QUALIFIER);
322
323 json = get.toJSON();
324 parsedJSON = mapper.readValue(json, HashMap.class);
325
326 assertEquals("row incorrect in Get.toJSON()",
327 Bytes.toStringBinary(ROW), parsedJSON.get("row"));
328
329 familyInfo = (List) ((Map) parsedJSON.get("families")).get(
330 Bytes.toStringBinary(FAMILY));
331 assertNotNull("Family absent in Get.toJSON()", familyInfo);
332 assertEquals("Qualifier absent in Get.toJSON()", 1, familyInfo.size());
333 assertEquals("Qualifier incorrect in Get.toJSON()",
334 Bytes.toStringBinary(QUALIFIER),
335 familyInfo.get(0));
336
337
338 Put put = new Put(ROW);
339 put.add(FAMILY, QUALIFIER, VALUE);
340
341 json = put.toJSON();
342 parsedJSON = mapper.readValue(json, HashMap.class);
343
344 assertEquals("row absent in Put.toJSON()",
345 Bytes.toStringBinary(ROW), parsedJSON.get("row"));
346
347 familyInfo = (List) ((Map) parsedJSON.get("families")).get(
348 Bytes.toStringBinary(FAMILY));
349 assertNotNull("Family absent in Put.toJSON()", familyInfo);
350 assertEquals("KeyValue absent in Put.toJSON()", 1, familyInfo.size());
351 Map kvMap = (Map) familyInfo.get(0);
352 assertEquals("Qualifier incorrect in Put.toJSON()",
353 Bytes.toStringBinary(QUALIFIER),
354 kvMap.get("qualifier"));
355 assertEquals("Value length incorrect in Put.toJSON()",
356 VALUE.length, kvMap.get("vlen"));
357
358
359 Delete delete = new Delete(ROW);
360 delete.deleteColumn(FAMILY, QUALIFIER);
361
362 json = delete.toJSON();
363 parsedJSON = mapper.readValue(json, HashMap.class);
364
365 assertEquals("row absent in Delete.toJSON()",
366 Bytes.toStringBinary(ROW), parsedJSON.get("row"));
367
368 familyInfo = (List) ((Map) parsedJSON.get("families")).get(
369 Bytes.toStringBinary(FAMILY));
370 assertNotNull("Family absent in Delete.toJSON()", familyInfo);
371 assertEquals("KeyValue absent in Delete.toJSON()", 1, familyInfo.size());
372 kvMap = (Map) familyInfo.get(0);
373 assertEquals("Qualifier incorrect in Delete.toJSON()",
374 Bytes.toStringBinary(QUALIFIER), kvMap.get("qualifier"));
375 }
376
377 @Test
378 public void testPutCreationWithByteBuffer() {
379 Put p = new Put(ROW);
380 List<Cell> c = p.get(FAMILY, QUALIFIER);
381 Assert.assertEquals(0, c.size());
382 Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
383
384 p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
385 c = p.get(FAMILY, QUALIFIER);
386 Assert.assertEquals(1, c.size());
387 Assert.assertEquals(1984L, c.get(0).getTimestamp());
388 Assert.assertArrayEquals(VALUE, CellUtil.cloneValue(c.get(0)));
389 Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
390 Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
391
392 p = new Put(ROW);
393 p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
394 c = p.get(FAMILY, QUALIFIER);
395 Assert.assertEquals(1, c.size());
396 Assert.assertEquals(2013L, c.get(0).getTimestamp());
397 Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
398 Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
399 Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
400
401 p = new Put(ByteBuffer.wrap(ROW));
402 p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
403 c = p.get(FAMILY, QUALIFIER);
404 Assert.assertEquals(1, c.size());
405 Assert.assertEquals(2001L, c.get(0).getTimestamp());
406 Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
407 Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
408 Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
409 Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
410
411 p = new Put(ByteBuffer.wrap(ROW), 1970L);
412 p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
413 c = p.get(FAMILY, QUALIFIER);
414 Assert.assertEquals(1, c.size());
415 Assert.assertEquals(2001L, c.get(0).getTimestamp());
416 Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
417 Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
418 Assert.assertEquals(1970L, p.getTimeStamp());
419 Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
420 }
421
422
423 }
424