1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.filter;
19  
20  import static org.junit.Assert.*;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.HashMap;
25  import java.util.HashSet;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.Set;
29  
30  import org.apache.hadoop.hbase.*;
31  import org.apache.hadoop.hbase.client.Put;
32  import org.apache.hadoop.hbase.client.Scan;
33  import org.apache.hadoop.hbase.regionserver.HRegion;
34  import org.apache.hadoop.hbase.regionserver.InternalScanner;
35  import org.apache.hadoop.hbase.util.Bytes;
36  import org.junit.Test;
37  import org.junit.experimental.categories.Category;
38  
39  @Category(SmallTests.class)
40  public class TestMultipleColumnPrefixFilter {
41  
42    private final static HBaseTestingUtility TEST_UTIL = new
43        HBaseTestingUtility();
44  
45    @Test
46    public void testMultipleColumnPrefixFilter() throws IOException {
47      String family = "Family";
48      HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
49      htd.addFamily(new HColumnDescriptor(family));
50      // HRegionInfo info = new HRegionInfo(htd, null, null, false);
51      HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
52      HRegion region = HRegion.createHRegion(info, TEST_UTIL.
53        getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
54  
55      List<String> rows = generateRandomWords(100, "row");
56      List<String> columns = generateRandomWords(10000, "column");
57      long maxTimestamp = 2;
58  
59      List<KeyValue> kvList = new ArrayList<KeyValue>();
60  
61      Map<String, List<KeyValue>> prefixMap = new HashMap<String,
62          List<KeyValue>>();
63  
64      prefixMap.put("p", new ArrayList<KeyValue>());
65      prefixMap.put("q", new ArrayList<KeyValue>());
66      prefixMap.put("s", new ArrayList<KeyValue>());
67  
68      String valueString = "ValueString";
69  
70      for (String row: rows) {
71        Put p = new Put(Bytes.toBytes(row));
72        p.setWriteToWAL(false);
73        for (String column: columns) {
74          for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
75            KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
76                valueString);
77            p.add(kv);
78            kvList.add(kv);
79            for (String s: prefixMap.keySet()) {
80              if (column.startsWith(s)) {
81                prefixMap.get(s).add(kv);
82              }
83            }
84          }
85        }
86        region.put(p);
87      }
88  
89      MultipleColumnPrefixFilter filter;
90      Scan scan = new Scan();
91      scan.setMaxVersions();
92      byte [][] filter_prefix = new byte [2][];
93      filter_prefix[0] = new byte [] {'p'};
94      filter_prefix[1] = new byte [] {'q'};
95      
96      filter = new MultipleColumnPrefixFilter(filter_prefix);
97      scan.setFilter(filter);
98      List<KeyValue> results = new ArrayList<KeyValue>();  
99      InternalScanner scanner = region.getScanner(scan);
100     while(scanner.next(results));
101     assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
102 
103     region.close();
104     region.getLog().closeAndDelete();
105   }
106 
107   @Test
108   public void testMultipleColumnPrefixFilterWithManyFamilies() throws IOException {
109     String family1 = "Family1";
110     String family2 = "Family2";
111     HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
112     htd.addFamily(new HColumnDescriptor(family1));
113     htd.addFamily(new HColumnDescriptor(family2));
114     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
115     HRegion region = HRegion.createHRegion(info, TEST_UTIL.
116       getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
117 
118     List<String> rows = generateRandomWords(100, "row");
119     List<String> columns = generateRandomWords(10000, "column");
120     long maxTimestamp = 3;
121 
122     List<KeyValue> kvList = new ArrayList<KeyValue>();
123 
124     Map<String, List<KeyValue>> prefixMap = new HashMap<String,
125         List<KeyValue>>();
126 
127     prefixMap.put("p", new ArrayList<KeyValue>());
128     prefixMap.put("q", new ArrayList<KeyValue>());
129     prefixMap.put("s", new ArrayList<KeyValue>());
130 
131     String valueString = "ValueString";
132 
133     for (String row: rows) {
134       Put p = new Put(Bytes.toBytes(row));
135       p.setWriteToWAL(false);
136       for (String column: columns) {
137         for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
138           double rand = Math.random();
139           KeyValue kv;
140           if (rand < 0.5) 
141             kv = KeyValueTestUtil.create(row, family1, column, timestamp,
142                 valueString);
143           else 
144             kv = KeyValueTestUtil.create(row, family2, column, timestamp,
145                 valueString);
146           p.add(kv);
147           kvList.add(kv);
148           for (String s: prefixMap.keySet()) {
149             if (column.startsWith(s)) {
150               prefixMap.get(s).add(kv);
151             }
152           }
153         }
154       }
155       region.put(p);
156     }
157 
158     MultipleColumnPrefixFilter filter;
159     Scan scan = new Scan();
160     scan.setMaxVersions();
161     byte [][] filter_prefix = new byte [2][];
162     filter_prefix[0] = new byte [] {'p'};
163     filter_prefix[1] = new byte [] {'q'};
164     
165     filter = new MultipleColumnPrefixFilter(filter_prefix);
166     scan.setFilter(filter);
167     List<KeyValue> results = new ArrayList<KeyValue>();  
168     InternalScanner scanner = region.getScanner(scan);
169     while(scanner.next(results));
170     assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
171 
172     region.close();
173     region.getLog().closeAndDelete();
174   }
175   
176   @Test
177   public void testMultipleColumnPrefixFilterWithColumnPrefixFilter() throws IOException {
178     String family = "Family";
179     HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
180     htd.addFamily(new HColumnDescriptor(family));
181     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
182     HRegion region = HRegion.createHRegion(info, TEST_UTIL.
183       getDataTestDir(), TEST_UTIL.getConfiguration(),htd);
184 
185     List<String> rows = generateRandomWords(100, "row");
186     List<String> columns = generateRandomWords(10000, "column");
187     long maxTimestamp = 2;
188 
189     String valueString = "ValueString";
190 
191     for (String row: rows) {
192       Put p = new Put(Bytes.toBytes(row));
193       p.setWriteToWAL(false);
194       for (String column: columns) {
195         for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
196           KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
197               valueString);
198           p.add(kv);
199         }
200       }
201       region.put(p);
202     }
203 
204     MultipleColumnPrefixFilter multiplePrefixFilter;
205     Scan scan1 = new Scan();
206     scan1.setMaxVersions();
207     byte [][] filter_prefix = new byte [1][];
208     filter_prefix[0] = new byte [] {'p'};
209  
210     multiplePrefixFilter = new MultipleColumnPrefixFilter(filter_prefix);
211     scan1.setFilter(multiplePrefixFilter);
212     List<KeyValue> results1 = new ArrayList<KeyValue>();  
213     InternalScanner scanner1 = region.getScanner(scan1);
214     while(scanner1.next(results1));
215     
216     ColumnPrefixFilter singlePrefixFilter;
217     Scan scan2 = new Scan();
218     scan2.setMaxVersions();
219     singlePrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("p"));
220  
221     scan2.setFilter(singlePrefixFilter);
222     List<KeyValue> results2 = new ArrayList<KeyValue>();  
223     InternalScanner scanner2 = region.getScanner(scan1);
224     while(scanner2.next(results2));
225     
226     assertEquals(results1.size(), results2.size());
227 
228     region.close();
229     region.getLog().closeAndDelete();
230   }
231   
232   List<String> generateRandomWords(int numberOfWords, String suffix) {
233     Set<String> wordSet = new HashSet<String>();
234     for (int i = 0; i < numberOfWords; i++) {
235       int lengthOfWords = (int) (Math.random()*2) + 1;
236       char[] wordChar = new char[lengthOfWords];
237       for (int j = 0; j < wordChar.length; j++) {
238         wordChar[j] = (char) (Math.random() * 26 + 97);
239       }
240       String word;
241       if (suffix == null) {
242         word = new String(wordChar);
243       } else {
244         word = new String(wordChar) + suffix;
245       }
246       wordSet.add(word);
247     }
248     List<String> wordList = new ArrayList<String>(wordSet);
249     return wordList;
250   }
251 
252   @org.junit.Rule
253   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
254     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
255 }
256 
257