1   /*
2    * Copyright 2010 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  
21  package org.apache.hadoop.hbase.filter;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.List;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.hbase.HBaseTestingUtility;
31  import org.apache.hadoop.hbase.HColumnDescriptor;
32  import org.apache.hadoop.hbase.HRegionInfo;
33  import org.apache.hadoop.hbase.HTableDescriptor;
34  import org.apache.hadoop.hbase.KeyValue;
35  import org.apache.hadoop.hbase.client.Put;
36  import org.apache.hadoop.hbase.client.Scan;
37  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
38  import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
39  import org.apache.hadoop.hbase.regionserver.HRegion;
40  import org.apache.hadoop.hbase.regionserver.InternalScanner;
41  import org.apache.hadoop.hbase.util.Bytes;
42  
43  import junit.framework.TestCase;
44  
45  public class TestDependentColumnFilter extends TestCase {
46    private final Log LOG = LogFactory.getLog(this.getClass());
47    private static final byte[][] ROWS = {
48  	  Bytes.toBytes("test1"),Bytes.toBytes("test2")
49    };
50    private static final byte[][] FAMILIES = {
51  	  Bytes.toBytes("familyOne"),Bytes.toBytes("familyTwo")
52    };
53    private static final long STAMP_BASE = System.currentTimeMillis();
54    private static final long[] STAMPS = {
55  	  STAMP_BASE-100, STAMP_BASE-200, STAMP_BASE-300
56    };
57    private static final byte[] QUALIFIER = Bytes.toBytes("qualifier");
58    private static final byte[][] BAD_VALS = {
59  	  Bytes.toBytes("bad1"), Bytes.toBytes("bad2"), Bytes.toBytes("bad3")
60    };
61    private static final byte[] MATCH_VAL = Bytes.toBytes("match");
62    private HBaseTestingUtility testUtil;
63  
64    List<KeyValue> testVals;
65    private HRegion region;
66  
67    @Override
68    protected void setUp() throws Exception {
69      super.setUp();
70  
71      testUtil = new HBaseTestingUtility();
72  
73      testVals = makeTestVals();
74  
75      HTableDescriptor htd = new HTableDescriptor(getName());
76      htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
77      htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
78      HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
79      this.region = HRegion.createHRegion(info, testUtil.getDataTestDir(),
80          testUtil.getConfiguration(), htd);
81      addData();
82    }
83  
84    @Override
85    protected void tearDown() throws Exception {
86      super.tearDown();
87      this.region.close();
88    }
89  
90    private void addData() throws IOException {
91      Put put = new Put(ROWS[0]);
92      // add in an entry for each stamp, with 2 as a "good" value
93      put.add(FAMILIES[0], QUALIFIER, STAMPS[0], BAD_VALS[0]);
94      put.add(FAMILIES[0], QUALIFIER, STAMPS[1], BAD_VALS[1]);
95      put.add(FAMILIES[0], QUALIFIER, STAMPS[2], MATCH_VAL);
96      // add in entries for stamps 0 and 2.
97      // without a value check both will be "accepted"
98      // with one 2 will be accepted(since the corresponding ts entry
99      // has a matching value
100     put.add(FAMILIES[1], QUALIFIER, STAMPS[0], BAD_VALS[0]);
101     put.add(FAMILIES[1], QUALIFIER, STAMPS[2], BAD_VALS[2]);
102 
103     this.region.put(put);
104 
105     put = new Put(ROWS[1]);
106     put.add(FAMILIES[0], QUALIFIER, STAMPS[0], BAD_VALS[0]);
107     // there is no corresponding timestamp for this so it should never pass
108     put.add(FAMILIES[0], QUALIFIER, STAMPS[2], MATCH_VAL);
109     // if we reverse the qualifiers this one should pass
110     put.add(FAMILIES[1], QUALIFIER, STAMPS[0], MATCH_VAL);
111     // should pass
112     put.add(FAMILIES[1], QUALIFIER, STAMPS[1], BAD_VALS[2]);
113 
114     this.region.put(put);
115   }
116 
117   private List<KeyValue> makeTestVals() {
118 	List<KeyValue> testVals = new ArrayList<KeyValue>();
119 	testVals.add(new KeyValue(ROWS[0], FAMILIES[0], QUALIFIER, STAMPS[0], BAD_VALS[0]));
120 	testVals.add(new KeyValue(ROWS[0], FAMILIES[0], QUALIFIER, STAMPS[1], BAD_VALS[1]));
121 	testVals.add(new KeyValue(ROWS[0], FAMILIES[1], QUALIFIER, STAMPS[1], BAD_VALS[2]));
122 	testVals.add(new KeyValue(ROWS[0], FAMILIES[1], QUALIFIER, STAMPS[0], MATCH_VAL));
123 	testVals.add(new KeyValue(ROWS[0], FAMILIES[1], QUALIFIER, STAMPS[2], BAD_VALS[2]));
124 
125 	return testVals;
126   }
127 
128   /**
129    * This shouldn't be confused with TestFilter#verifyScan
130    * as expectedKeys is not the per row total, but the scan total
131    *
132    * @param s
133    * @param expectedRows
134    * @param expectedCells
135    * @throws IOException
136    */
137   private void verifyScan(Scan s, long expectedRows, long expectedCells)
138   throws IOException {
139     InternalScanner scanner = this.region.getScanner(s);
140     List<KeyValue> results = new ArrayList<KeyValue>();
141     int i = 0;
142     int cells = 0;
143     for (boolean done = true; done; i++) {
144       done = scanner.next(results);
145       Arrays.sort(results.toArray(new KeyValue[results.size()]),
146           KeyValue.COMPARATOR);
147       LOG.info("counter=" + i + ", " + results);
148       if (results.isEmpty()) break;
149       cells += results.size();
150       assertTrue("Scanned too many rows! Only expected " + expectedRows +
151           " total but already scanned " + (i+1), expectedRows > i);
152       assertTrue("Expected " + expectedCells + " cells total but " +
153           "already scanned " + cells, expectedCells >= cells);
154       results.clear();
155     }
156     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
157         " rows", expectedRows, i);
158     assertEquals("Expected " + expectedCells + " cells but scanned " + cells +
159             " cells", expectedCells, cells);
160   }
161 
162   /**
163    * Test scans using a DependentColumnFilter
164    */
165   public void testScans() throws Exception {
166     Filter filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER);
167 
168     Scan scan = new Scan();
169     scan.setFilter(filter);
170     scan.setMaxVersions(Integer.MAX_VALUE);
171 
172     verifyScan(scan, 2, 8);
173 
174     // drop the filtering cells
175     filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER, true);
176     scan = new Scan();
177     scan.setFilter(filter);
178     scan.setMaxVersions(Integer.MAX_VALUE);
179 
180     verifyScan(scan, 2, 3);
181 
182     // include a comparator operation
183     filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER, false,
184         CompareOp.EQUAL, new BinaryComparator(MATCH_VAL));
185     scan = new Scan();
186     scan.setFilter(filter);
187     scan.setMaxVersions(Integer.MAX_VALUE);
188 
189     /*
190      * expecting to get the following 3 cells
191      * row 0
192      *   put.add(FAMILIES[0], QUALIFIER, STAMPS[2], MATCH_VAL);
193      *   put.add(FAMILIES[1], QUALIFIER, STAMPS[2], BAD_VALS[2]);
194      * row 1
195      *   put.add(FAMILIES[0], QUALIFIER, STAMPS[2], MATCH_VAL);
196      */
197     verifyScan(scan, 2, 3);
198 
199     // include a comparator operation and drop comparator
200     filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER, true,
201         CompareOp.EQUAL, new BinaryComparator(MATCH_VAL));
202     scan = new Scan();
203     scan.setFilter(filter);
204     scan.setMaxVersions(Integer.MAX_VALUE);
205 
206     /*
207      * expecting to get the following 1 cell
208      * row 0
209      *   put.add(FAMILIES[1], QUALIFIER, STAMPS[2], BAD_VALS[2]);
210      */
211     verifyScan(scan, 1, 1);
212 
213   }
214 
215   /**
216    * Test that the filter correctly drops rows without a corresponding timestamp
217    *
218    * @throws Exception
219    */
220   public void testFilterDropping() throws Exception {
221     Filter filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER);
222     List<KeyValue> accepted = new ArrayList<KeyValue>();
223     for(KeyValue val : testVals) {
224       if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) {
225         accepted.add(val);
226       }
227     }
228     assertEquals("check all values accepted from filterKeyValue", 5, accepted.size());
229 
230     filter.filterRow(accepted);
231     assertEquals("check filterRow(List<KeyValue>) dropped cell without corresponding column entry", 4, accepted.size());
232 
233     // start do it again with dependent column dropping on
234     filter = new DependentColumnFilter(FAMILIES[1], QUALIFIER, true);
235     accepted.clear();
236     for(KeyValue val : testVals) {
237         if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) {
238           accepted.add(val);
239         }
240       }
241       assertEquals("check the filtering column cells got dropped", 2, accepted.size());
242 
243       filter.filterRow(accepted);
244       assertEquals("check cell retention", 2, accepted.size());
245   }
246 }