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.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.KeyValueTestUtil;
36  import org.apache.hadoop.hbase.client.Put;
37  import org.apache.hadoop.hbase.client.Scan;
38  import org.apache.hadoop.hbase.regionserver.HRegion;
39  import org.apache.hadoop.hbase.regionserver.InternalScanner;
40  import org.apache.hadoop.hbase.util.Bytes;
41  import org.junit.Test;
42  
43  public class TestMultipleColumnPrefixFilter {
44  
45    private final static HBaseTestingUtility TEST_UTIL = new
46        HBaseTestingUtility();
47  
48    @Test
49    public void testMultipleColumnPrefixFilter() throws IOException {
50      String family = "Family";
51      HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
52      htd.addFamily(new HColumnDescriptor(family));
53      // HRegionInfo info = new HRegionInfo(htd, null, null, false);
54      HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
55      HRegion region = HRegion.createHRegion(info, TEST_UTIL.
56        getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
57  
58      List<String> rows = generateRandomWords(100, "row");
59      List<String> columns = generateRandomWords(10000, "column");
60      long maxTimestamp = 2;
61  
62      List<KeyValue> kvList = new ArrayList<KeyValue>();
63  
64      Map<String, List<KeyValue>> prefixMap = new HashMap<String,
65          List<KeyValue>>();
66  
67      prefixMap.put("p", new ArrayList<KeyValue>());
68      prefixMap.put("q", new ArrayList<KeyValue>());
69      prefixMap.put("s", new ArrayList<KeyValue>());
70  
71      String valueString = "ValueString";
72  
73      for (String row: rows) {
74        Put p = new Put(Bytes.toBytes(row));
75        p.setWriteToWAL(false);
76        for (String column: columns) {
77          for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
78            KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
79                valueString);
80            p.add(kv);
81            kvList.add(kv);
82            for (String s: prefixMap.keySet()) {
83              if (column.startsWith(s)) {
84                prefixMap.get(s).add(kv);
85              }
86            }
87          }
88        }
89        region.put(p);
90      }
91  
92      MultipleColumnPrefixFilter filter;
93      Scan scan = new Scan();
94      scan.setMaxVersions();
95      byte [][] filter_prefix = new byte [2][];
96      filter_prefix[0] = new byte [] {'p'};
97      filter_prefix[1] = new byte [] {'q'};
98      
99      filter = new MultipleColumnPrefixFilter(filter_prefix);
100     scan.setFilter(filter);
101     List<KeyValue> results = new ArrayList<KeyValue>();  
102     InternalScanner scanner = region.getScanner(scan);
103     while(scanner.next(results));
104     assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
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   
173   @Test
174   public void testMultipleColumnPrefixFilterWithColumnPrefixFilter() throws IOException {
175     String family = "Family";
176     HTableDescriptor htd = new HTableDescriptor("TestMultipleColumnPrefixFilter");
177     htd.addFamily(new HColumnDescriptor(family));
178     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
179     HRegion region = HRegion.createHRegion(info, TEST_UTIL.
180       getDataTestDir(), TEST_UTIL.getConfiguration(),htd);
181 
182     List<String> rows = generateRandomWords(100, "row");
183     List<String> columns = generateRandomWords(10000, "column");
184     long maxTimestamp = 2;
185 
186     String valueString = "ValueString";
187 
188     for (String row: rows) {
189       Put p = new Put(Bytes.toBytes(row));
190       p.setWriteToWAL(false);
191       for (String column: columns) {
192         for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
193           KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
194               valueString);
195           p.add(kv);
196         }
197       }
198       region.put(p);
199     }
200 
201     MultipleColumnPrefixFilter multiplePrefixFilter;
202     Scan scan1 = new Scan();
203     scan1.setMaxVersions();
204     byte [][] filter_prefix = new byte [1][];
205     filter_prefix[0] = new byte [] {'p'};
206  
207     multiplePrefixFilter = new MultipleColumnPrefixFilter(filter_prefix);
208     scan1.setFilter(multiplePrefixFilter);
209     List<KeyValue> results1 = new ArrayList<KeyValue>();  
210     InternalScanner scanner1 = region.getScanner(scan1);
211     while(scanner1.next(results1));
212     
213     ColumnPrefixFilter singlePrefixFilter;
214     Scan scan2 = new Scan();
215     scan2.setMaxVersions();
216     singlePrefixFilter = new ColumnPrefixFilter(Bytes.toBytes("p"));
217  
218     scan2.setFilter(singlePrefixFilter);
219     List<KeyValue> results2 = new ArrayList<KeyValue>();  
220     InternalScanner scanner2 = region.getScanner(scan1);
221     while(scanner2.next(results2));
222     
223     assertEquals(results1.size(), results2.size());
224   }
225   
226   List<String> generateRandomWords(int numberOfWords, String suffix) {
227     Set<String> wordSet = new HashSet<String>();
228     for (int i = 0; i < numberOfWords; i++) {
229       int lengthOfWords = (int) (Math.random()*2) + 1;
230       char[] wordChar = new char[lengthOfWords];
231       for (int j = 0; j < wordChar.length; j++) {
232         wordChar[j] = (char) (Math.random() * 26 + 97);
233       }
234       String word;
235       if (suffix == null) {
236         word = new String(wordChar);
237       } else {
238         word = new String(wordChar) + suffix;
239       }
240       wordSet.add(word);
241     }
242     List<String> wordList = new ArrayList<String>(wordSet);
243     return wordList;
244   }
245 }
246