1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.filter;
21  
22  import static org.junit.Assert.assertEquals;
23  
24  import java.io.ByteArrayInputStream;
25  import java.io.ByteArrayOutputStream;
26  import java.io.DataInput;
27  import java.io.DataInputStream;
28  import java.io.DataOutput;
29  import java.io.DataOutputStream;
30  import java.io.IOException;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.List;
34  
35  import junit.framework.TestCase;
36  
37  import org.apache.hadoop.hbase.KeyValue;
38  import org.apache.hadoop.hbase.SmallTests;
39  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
40  import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
41  import org.apache.hadoop.hbase.filter.FilterList.Operator;
42  import org.apache.hadoop.hbase.util.Bytes;
43  import org.junit.experimental.categories.Category;
44  
45  import com.google.common.collect.Lists;
46  
47  /**
48   * Tests filter sets
49   *
50   */
51  @Category(SmallTests.class)
52  public class TestFilterList extends TestCase {
53    static final int MAX_PAGES = 2;
54    static final char FIRST_CHAR = 'a';
55    static final char LAST_CHAR = 'e';
56    static byte[] GOOD_BYTES = Bytes.toBytes("abc");
57    static byte[] BAD_BYTES = Bytes.toBytes("def");
58  
59    /**
60     * Test "must pass one"
61     * @throws Exception
62     */
63    public void testMPONE() throws Exception {
64      List<Filter> filters = new ArrayList<Filter>();
65      filters.add(new PageFilter(MAX_PAGES));
66      filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
67      Filter filterMPONE =
68          new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
69      /* Filter must do all below steps:
70       * <ul>
71       * <li>{@link #reset()}</li>
72       * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
73       * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
74       * if false, we will also call</li>
75       * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
76       * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
77       * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
78       * </li>
79       * </ul>
80      */
81      filterMPONE.reset();
82      assertFalse(filterMPONE.filterAllRemaining());
83  
84      /* Will pass both */
85      byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
86      for (int i = 0; i < MAX_PAGES - 1; i++) {
87        assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
88        KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
89          Bytes.toBytes(i));
90        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
91        assertFalse(filterMPONE.filterRow());
92      }
93  
94      /* Only pass PageFilter */
95      rowkey = Bytes.toBytes("z");
96      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
97      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
98          Bytes.toBytes(0));
99      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
100     assertFalse(filterMPONE.filterRow());
101 
102     /* reach MAX_PAGES already, should filter any rows */
103     rowkey = Bytes.toBytes("yyy");
104     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
105     kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
106         Bytes.toBytes(0));
107     assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
108 
109     /* We should filter any row */
110     rowkey = Bytes.toBytes("z");
111     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
112     assertTrue(filterMPONE.filterAllRemaining());
113   }
114 
115   /**
116    * Test "must pass all"
117    * @throws Exception
118    */
119   public void testMPALL() throws Exception {
120     List<Filter> filters = new ArrayList<Filter>();
121     filters.add(new PageFilter(MAX_PAGES));
122     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
123     Filter filterMPALL =
124       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
125     /* Filter must do all below steps:
126      * <ul>
127      * <li>{@link #reset()}</li>
128      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
129      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
130      * if false, we will also call</li>
131      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
132      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
133      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
134      * </li>
135      * </ul>
136     */
137     filterMPALL.reset();
138     assertFalse(filterMPALL.filterAllRemaining());
139     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
140     for (int i = 0; i < MAX_PAGES - 1; i++) {
141       assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
142       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
143         Bytes.toBytes(i));
144       assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
145     }
146     filterMPALL.reset();
147     rowkey = Bytes.toBytes("z");
148     assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
149     // Should fail here; row should be filtered out.
150     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
151     assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
152   }
153 
154   /**
155    * Test list ordering
156    * @throws Exception
157    */
158   public void testOrdering() throws Exception {
159     List<Filter> filters = new ArrayList<Filter>();
160     filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
161     filters.add(new PageFilter(MAX_PAGES));
162     Filter filterMPONE =
163         new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
164     /* Filter must do all below steps:
165      * <ul>
166      * <li>{@link #reset()}</li>
167      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
168      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
169      * if false, we will also call</li>
170      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
171      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
172      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
173      * </li>
174      * </ul>
175     */
176     filterMPONE.reset();
177     assertFalse(filterMPONE.filterAllRemaining());
178 
179     /* We should be able to fill MAX_PAGES without incrementing page counter */
180     byte [] rowkey = Bytes.toBytes("yyyyyyyy");
181     for (int i = 0; i < MAX_PAGES; i++) {
182       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
183       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
184           Bytes.toBytes(i));
185         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
186       assertFalse(filterMPONE.filterRow());
187     }
188 
189     /* Now let's fill the page filter */
190     rowkey = Bytes.toBytes("xxxxxxx");
191     for (int i = 0; i < MAX_PAGES; i++) {
192       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
193       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
194           Bytes.toBytes(i));
195         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
196       assertFalse(filterMPONE.filterRow());
197     }
198 
199     /* We should still be able to include even though page filter is at max */
200     rowkey = Bytes.toBytes("yyy");
201     for (int i = 0; i < MAX_PAGES; i++) {
202       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
203       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
204           Bytes.toBytes(i));
205         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
206       assertFalse(filterMPONE.filterRow());
207     }
208   }
209 
210   private static class AlwaysNextColFilter extends FilterBase {
211     public AlwaysNextColFilter() {
212       super();
213     }
214     @Override
215     public ReturnCode filterKeyValue(KeyValue v) {
216       return ReturnCode.NEXT_COL;
217     }
218     @Override
219     public void readFields(DataInput arg0) throws IOException {}
220 
221     @Override
222     public void write(DataOutput arg0) throws IOException {}
223   }
224 
225   /**
226    * When we do a "MUST_PASS_ONE" (a logical 'OR') of the two filters
227    * we expect to get the same result as the inclusive stop result.
228    * @throws Exception
229    */
230   public void testFilterListWithInclusiveStopFilteMustPassOne() throws Exception {
231     byte[] r1 = Bytes.toBytes("Row1");
232     byte[] r11 = Bytes.toBytes("Row11");
233     byte[] r2 = Bytes.toBytes("Row2");
234 
235     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
236     flist.addFilter(new AlwaysNextColFilter());
237     flist.addFilter(new InclusiveStopFilter(r1));
238     flist.filterRowKey(r1, 0, r1.length);
239     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
240     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
241 
242     flist.reset();
243     flist.filterRowKey(r2, 0, r2.length);
244     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
245   }
246 
247   /**
248    * When we do a "MUST_PASS_ONE" (a logical 'OR') of the above two filters
249    * we expect to get the same result as the 'prefix' only result.
250    * @throws Exception
251    */
252   public void testFilterListTwoFiltersMustPassOne() throws Exception {
253   byte[] r1 = Bytes.toBytes("Row1");
254     byte[] r11 = Bytes.toBytes("Row11");
255     byte[] r2 = Bytes.toBytes("Row2");
256 
257     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
258     flist.addFilter(new PrefixFilter(r1));
259     flist.filterRowKey(r1, 0, r1.length);
260     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
261     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
262 
263     flist.reset();
264     flist.filterRowKey(r2, 0, r2.length);
265     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
266 
267     flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
268     flist.addFilter(new AlwaysNextColFilter());
269     flist.addFilter(new PrefixFilter(r1));
270     flist.filterRowKey(r1, 0, r1.length);
271     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
272     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
273 
274     flist.reset();
275     flist.filterRowKey(r2, 0, r2.length);
276     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
277   }
278 
279   /**
280    * Test serialization
281    * @throws Exception
282    */
283   public void testSerialization() throws Exception {
284     List<Filter> filters = new ArrayList<Filter>();
285     filters.add(new PageFilter(MAX_PAGES));
286     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
287     Filter filterMPALL =
288       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
289 
290     // Decompose filterMPALL to bytes.
291     ByteArrayOutputStream stream = new ByteArrayOutputStream();
292     DataOutputStream out = new DataOutputStream(stream);
293     filterMPALL.write(out);
294     out.close();
295     byte[] buffer = stream.toByteArray();
296 
297     // Recompose filterMPALL.
298     DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
299     FilterList newFilter = new FilterList();
300     newFilter.readFields(in);
301 
302     // TODO: Run TESTS!!!
303   }
304 
305   /**
306    * Test filterKeyValue logic.
307    * @throws Exception
308    */
309   public void testFilterKeyValue() throws Exception {
310     Filter includeFilter = new FilterBase() {
311       @Override
312       public Filter.ReturnCode filterKeyValue(KeyValue v) {
313         return Filter.ReturnCode.INCLUDE;
314       }
315 
316       @Override
317       public void readFields(DataInput arg0) throws IOException {}
318 
319       @Override
320       public void write(DataOutput arg0) throws IOException {}
321     };
322 
323     Filter alternateFilter = new FilterBase() {
324       boolean returnInclude = true;
325 
326       @Override
327       public Filter.ReturnCode filterKeyValue(KeyValue v) {
328         Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
329                                                        Filter.ReturnCode.SKIP;
330         returnInclude = !returnInclude;
331         return returnCode;
332       }
333 
334       @Override
335       public void readFields(DataInput arg0) throws IOException {}
336 
337       @Override
338       public void write(DataOutput arg0) throws IOException {}
339     };
340 
341     Filter alternateIncludeFilter = new FilterBase() {
342       boolean returnIncludeOnly = false;
343 
344       @Override
345       public Filter.ReturnCode filterKeyValue(KeyValue v) {
346         Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
347                                                            Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
348         returnIncludeOnly = !returnIncludeOnly;
349         return returnCode;
350       }
351 
352       @Override
353       public void readFields(DataInput arg0) throws IOException {}
354 
355       @Override
356       public void write(DataOutput arg0) throws IOException {}
357     };
358 
359     // Check must pass one filter.
360     FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
361         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
362     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
363     assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpOnefilterList.filterKeyValue(null));
364     // INCLUDE, SKIP, INCLUDE. 
365     assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
366 
367     // Check must pass all filter.
368     FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
369         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
370     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
371     assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
372     // INCLUDE, SKIP, INCLUDE. 
373     assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
374   }
375 
376   /**
377    * Test pass-thru of hints.
378    */
379   public void testHintPassThru() throws Exception {
380 
381     final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
382     final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
383         null, null);
384 
385     Filter filterNoHint = new FilterBase() {
386       @Override
387       public void readFields(DataInput arg0) throws IOException {}
388 
389       @Override
390       public void write(DataOutput arg0) throws IOException {}
391     };
392 
393     Filter filterMinHint = new FilterBase() {
394       @Override
395       public ReturnCode filterKeyValue(KeyValue ignored) {
396         return ReturnCode.SEEK_NEXT_USING_HINT;
397       }
398 
399       @Override
400       public KeyValue getNextKeyHint(KeyValue currentKV) {
401         return minKeyValue;
402       }
403 
404       @Override
405       public void readFields(DataInput arg0) throws IOException {}
406 
407       @Override
408       public void write(DataOutput arg0) throws IOException {}
409     };
410 
411     Filter filterMaxHint = new FilterBase() {
412       @Override
413       public ReturnCode filterKeyValue(KeyValue ignored) {
414         return ReturnCode.SEEK_NEXT_USING_HINT;
415       }
416 
417       @Override
418       public KeyValue getNextKeyHint(KeyValue currentKV) {
419         return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
420       }
421 
422       @Override
423       public void readFields(DataInput arg0) throws IOException {}
424 
425       @Override
426       public void write(DataOutput arg0) throws IOException {}
427     };
428 
429     // MUST PASS ONE
430 
431     // Should take the min if given two hints
432     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
433         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
434     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
435         minKeyValue));
436 
437     // Should have no hint if any filter has no hint
438     filterList = new FilterList(Operator.MUST_PASS_ONE,
439         Arrays.asList(
440             new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
441     assertNull(filterList.getNextKeyHint(null));
442     filterList = new FilterList(Operator.MUST_PASS_ONE,
443         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
444     assertNull(filterList.getNextKeyHint(null));
445 
446     // Should give max hint if its the only one
447     filterList = new FilterList(Operator.MUST_PASS_ONE,
448         Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
449     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
450         maxKeyValue));
451 
452     // MUST PASS ALL
453 
454     // Should take the first hint
455     filterList = new FilterList(Operator.MUST_PASS_ALL,
456         Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
457     filterList.filterKeyValue(null);
458     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
459         minKeyValue));
460 
461     filterList = new FilterList(Operator.MUST_PASS_ALL,
462         Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
463     filterList.filterKeyValue(null);
464     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
465         maxKeyValue));
466 
467     // Should have first hint even if a filter has no hint
468     filterList = new FilterList(Operator.MUST_PASS_ALL,
469         Arrays.asList(
470             new Filter [] { filterNoHint, filterMinHint, filterMaxHint } ));
471     filterList.filterKeyValue(null);
472     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
473         minKeyValue));
474     filterList = new FilterList(Operator.MUST_PASS_ALL,
475         Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
476     filterList.filterKeyValue(null);
477     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
478         maxKeyValue));
479     filterList.filterKeyValue(null);
480     filterList = new FilterList(Operator.MUST_PASS_ALL,
481         Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
482     filterList.filterKeyValue(null);
483     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
484         minKeyValue));
485   }
486 
487   /**
488    * Tests the behavior of transform() in a hierarchical filter.
489    *
490    * transform() only applies after a filterKeyValue() whose return-code includes the KeyValue.
491    * Lazy evaluation of AND
492    */
493   public void testTransformMPO() throws Exception {
494     // Apply the following filter:
495     //     (family=fam AND qualifier=qual1 AND KeyOnlyFilter)
496     //  OR (family=fam AND qualifier=qual2)
497     final FilterList flist = new FilterList(Operator.MUST_PASS_ONE, Lists.<Filter>newArrayList(
498         new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
499             new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
500             new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual1"))),
501             new KeyOnlyFilter())),
502         new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
503             new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
504             new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual2")))))));
505 
506     final KeyValue kvQual1 = new KeyValue(
507         Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual1"), Bytes.toBytes("value"));
508     final KeyValue kvQual2 = new KeyValue(
509         Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual2"), Bytes.toBytes("value"));
510     final KeyValue kvQual3 = new KeyValue(
511         Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
512 
513     // Value for fam:qual1 should be stripped:
514     assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
515     final KeyValue transformedQual1 = flist.transform(kvQual1);
516     assertEquals(0, transformedQual1.getValue().length);
517 
518     // Value for fam:qual2 should not be stripped:
519     assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
520     final KeyValue transformedQual2 = flist.transform(kvQual2);
521     assertEquals("value", Bytes.toString(transformedQual2.getValue()));
522 
523     // Other keys should be skipped:
524     assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
525   }
526 
527   @org.junit.Rule
528   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
529     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
530 }
531