View Javadoc

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  
19  package org.apache.hadoop.hbase.codec.prefixtree.row;
20  
21  import java.io.ByteArrayOutputStream;
22  import java.io.IOException;
23  import java.nio.ByteBuffer;
24  import java.util.Collection;
25  import java.util.List;
26  
27  import org.apache.hadoop.hbase.Cell;
28  import org.apache.hadoop.hbase.CellComparator;
29  import org.apache.hadoop.hbase.KeyValue;
30  import org.apache.hadoop.hbase.KeyValueUtil;
31  import org.apache.hadoop.hbase.SmallTests;
32  import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
33  import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
34  import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
35  import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
36  import org.apache.hadoop.hbase.util.CollectionUtils;
37  import org.junit.Assert;
38  import org.junit.Test;
39  import org.junit.experimental.categories.Category;
40  import org.junit.runner.RunWith;
41  import org.junit.runners.Parameterized;
42  import org.junit.runners.Parameterized.Parameters;
43  
44  @Category(SmallTests.class)
45  @RunWith(Parameterized.class)
46  public class TestPrefixTreeSearcher {
47  
48  	protected static int BLOCK_START = 7;
49  
50    @Parameters
51    public static Collection<Object[]> parameters() {
52      return new TestRowData.InMemory().getAllAsObjectArray();
53    }
54  
55    protected TestRowData rows;
56    protected ByteBuffer block;
57  
58    public TestPrefixTreeSearcher(TestRowData testRows) throws IOException {
59      this.rows = testRows;
60      ByteArrayOutputStream os = new ByteArrayOutputStream(1 << 20);
61      PrefixTreeEncoder kvBuilder = new PrefixTreeEncoder(os, true);
62      for (KeyValue kv : rows.getInputs()) {
63        kvBuilder.write(kv);
64      }
65      kvBuilder.flush();
66      byte[] outputBytes = os.toByteArray();
67      this.block = ByteBuffer.wrap(outputBytes);
68    }
69  
70  
71    @Test
72    public void testScanForwards() throws IOException {
73      CellSearcher searcher = null;
74      try {
75        searcher = DecoderFactory.checkOut(block, true);
76  
77        int i = -1;
78        while (searcher.advance()) {
79          ++i;
80          KeyValue inputCell = rows.getInputs().get(i);
81          Cell outputCell = searcher.current();
82  
83          // check all 3 permutations of equals()
84          Assert.assertEquals(inputCell, outputCell);
85          Assert.assertEquals(outputCell, inputCell);
86          Assert.assertTrue(CellComparator.equals(inputCell, outputCell));
87        }
88        Assert.assertEquals(rows.getInputs().size(), i + 1);
89      } finally {
90        DecoderFactory.checkIn(searcher);
91      }
92    }
93  
94  
95    @Test
96    public void testScanBackwards() throws IOException {
97      CellSearcher searcher = null;
98      try {
99        searcher = DecoderFactory.checkOut(block, true);
100       searcher.positionAfterLastCell();
101       int i = -1;
102       while (searcher.previous()) {
103         ++i;
104         int oppositeIndex = rows.getInputs().size() - i - 1;
105         KeyValue inputKv = rows.getInputs().get(oppositeIndex);
106         KeyValue outputKv = KeyValueUtil.copyToNewKeyValue(searcher.current());
107         Assert.assertEquals(inputKv, outputKv);
108       }
109       Assert.assertEquals(rows.getInputs().size(), i + 1);
110     } finally {
111       DecoderFactory.checkIn(searcher);
112     }
113   }
114 
115 
116   @Test
117   public void testRandomSeekHits() throws IOException {
118     CellSearcher searcher = null;
119     try {
120       searcher = DecoderFactory.checkOut(block, true);
121       for (KeyValue kv : rows.getInputs()) {
122         boolean hit = searcher.positionAt(kv);
123         Assert.assertTrue(hit);
124         Cell foundKv = searcher.current();
125         Assert.assertTrue(CellComparator.equals(kv, foundKv));
126       }
127     } finally {
128       DecoderFactory.checkIn(searcher);
129     }
130   }
131 
132   @Test
133   public void testRandomSeekMisses() throws IOException {
134     CellSearcher searcher = null;
135     List<Integer> rowStartIndexes = rows.getRowStartIndexes();
136     try {
137       searcher = DecoderFactory.checkOut(block, true);
138 
139       //test both the positionAtOrBefore and positionAtOrAfter methods
140       for(boolean beforeVsAfterOnMiss : new boolean[]{true, false}){
141         for (int i=0; i < rows.getInputs().size(); ++i) {
142           KeyValue kv = rows.getInputs().get(i);
143 
144           //nextRow
145           KeyValue inputNextRow = KeyValueUtil.createFirstKeyInNextRow(kv);
146 
147           CellScannerPosition position = beforeVsAfterOnMiss
148               ? searcher.positionAtOrBefore(inputNextRow)
149               : searcher.positionAtOrAfter(inputNextRow);
150           boolean isFirstInRow = rowStartIndexes.contains(i);
151           if(isFirstInRow){
152             int rowIndex = rowStartIndexes.indexOf(i);
153             if(rowIndex < rowStartIndexes.size() - 1){
154               if(beforeVsAfterOnMiss){
155                 Assert.assertEquals(CellScannerPosition.BEFORE, position);
156               }else{
157                 Assert.assertEquals(CellScannerPosition.AFTER, position);
158               }
159 
160               int expectedInputIndex = beforeVsAfterOnMiss
161                   ? rowStartIndexes.get(rowIndex + 1) - 1
162                   : rowStartIndexes.get(rowIndex + 1);
163               Assert.assertEquals(rows.getInputs().get(expectedInputIndex), searcher.current());
164             }
165           }
166 
167           //previous KV
168           KeyValue inputPreviousKv = KeyValueUtil.previousKey(kv);
169           boolean hit = searcher.positionAt(inputPreviousKv);
170           Assert.assertFalse(hit);
171           position = searcher.positionAtOrAfter(inputPreviousKv);
172           if(CollectionUtils.isLastIndex(rows.getInputs(), i)){
173             Assert.assertTrue(CellScannerPosition.AFTER_LAST == position);
174           }else{
175             Assert.assertTrue(CellScannerPosition.AFTER == position);
176             /*
177              * TODO: why i+1 instead of i?
178              */
179             Assert.assertEquals(rows.getInputs().get(i+1), searcher.current());
180           }
181         }
182       }
183     } finally {
184       DecoderFactory.checkIn(searcher);
185     }
186   }
187 
188 
189   @Test
190   public void testRandomSeekIndividualAssertions() throws IOException {
191     CellSearcher searcher = null;
192     try {
193       searcher = DecoderFactory.checkOut(block, true);
194       rows.individualSearcherAssertions(searcher);
195     } finally {
196       DecoderFactory.checkIn(searcher);
197     }
198   }
199 }