001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver.querymatcher;
019
020import static org.junit.Assert.assertArrayEquals;
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertFalse;
023
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.List;
027import org.apache.hadoop.hbase.Cell;
028import org.apache.hadoop.hbase.HBaseClassTestRule;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.KeepDeletedCells;
031import org.apache.hadoop.hbase.KeyValue;
032import org.apache.hadoop.hbase.PrivateCellUtil;
033import org.apache.hadoop.hbase.client.Scan;
034import org.apache.hadoop.hbase.filter.FilterBase;
035import org.apache.hadoop.hbase.regionserver.ScanInfo;
036import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode;
037import org.apache.hadoop.hbase.testclassification.RegionServerTests;
038import org.apache.hadoop.hbase.testclassification.SmallTests;
039import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
040import org.junit.ClassRule;
041import org.junit.Test;
042import org.junit.experimental.categories.Category;
043import org.slf4j.Logger;
044import org.slf4j.LoggerFactory;
045
046@Category({ RegionServerTests.class, SmallTests.class })
047public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher {
048
049  @ClassRule
050  public static final HBaseClassTestRule CLASS_RULE =
051    HBaseClassTestRule.forClass(TestUserScanQueryMatcher.class);
052
053  private static final Logger LOG = LoggerFactory.getLogger(TestUserScanQueryMatcher.class);
054
055  /**
056   * This is a cryptic test. It is checking that we don't include a fake cell, one that has a
057   * timestamp of {@link HConstants#OLDEST_TIMESTAMP}. See HBASE-16074 for background. n
058   */
059  @Test
060  public void testNeverIncludeFakeCell() throws IOException {
061    long now = EnvironmentEdgeManager.currentTime();
062    // Do with fam2 which has a col2 qualifier.
063    UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan,
064      new ScanInfo(this.conf, fam2, 10, 1, ttl, KeepDeletedCells.FALSE,
065        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
066      get.getFamilyMap().get(fam2), now - ttl, now, null);
067    Cell kv = new KeyValue(row1, fam2, col2, 1, data);
068    Cell cell = PrivateCellUtil.createLastOnRowCol(kv);
069    qm.setToNewRow(kv);
070    MatchCode code = qm.match(cell);
071    assertFalse(code.compareTo(MatchCode.SEEK_NEXT_COL) != 0);
072  }
073
074  @Test
075  public void testMatchExplicitColumns() throws IOException {
076    // Moving up from the Tracker by using Gets and List<KeyValue> instead
077    // of just byte []
078
079    // Expected result
080    List<MatchCode> expected = new ArrayList<>(6);
081    expected.add(ScanQueryMatcher.MatchCode.SEEK_NEXT_COL);
082    expected.add(ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL);
083    expected.add(ScanQueryMatcher.MatchCode.SEEK_NEXT_COL);
084    expected.add(ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL);
085    expected.add(ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW);
086    expected.add(ScanQueryMatcher.MatchCode.DONE);
087
088    long now = EnvironmentEdgeManager.currentTime();
089    // 2,4,5
090    UserScanQueryMatcher qm = UserScanQueryMatcher.create(
091      scan, new ScanInfo(this.conf, fam2, 0, 1, ttl, KeepDeletedCells.FALSE,
092        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
093      get.getFamilyMap().get(fam2), now - ttl, now, null);
094
095    List<KeyValue> memstore = new ArrayList<>(6);
096    memstore.add(new KeyValue(row1, fam2, col1, 1, data));
097    memstore.add(new KeyValue(row1, fam2, col2, 1, data));
098    memstore.add(new KeyValue(row1, fam2, col3, 1, data));
099    memstore.add(new KeyValue(row1, fam2, col4, 1, data));
100    memstore.add(new KeyValue(row1, fam2, col5, 1, data));
101
102    memstore.add(new KeyValue(row2, fam1, col1, data));
103
104    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<>(memstore.size());
105    KeyValue k = memstore.get(0);
106    qm.setToNewRow(k);
107
108    for (KeyValue kv : memstore) {
109      actual.add(qm.match(kv));
110    }
111
112    assertEquals(expected.size(), actual.size());
113    for (int i = 0; i < expected.size(); i++) {
114      LOG.debug("expected " + expected.get(i) + ", actual " + actual.get(i));
115      assertEquals(expected.get(i), actual.get(i));
116    }
117  }
118
119  @Test
120  public void testMatch_Wildcard() throws IOException {
121    // Moving up from the Tracker by using Gets and List<KeyValue> instead
122    // of just byte []
123
124    // Expected result
125    List<MatchCode> expected = new ArrayList<>(6);
126    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
127    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
128    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
129    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
130    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
131    expected.add(ScanQueryMatcher.MatchCode.DONE);
132
133    long now = EnvironmentEdgeManager.currentTime();
134    UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1,
135      ttl, KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false), null,
136      now - ttl, now, null);
137
138    List<KeyValue> memstore = new ArrayList<>(6);
139    memstore.add(new KeyValue(row1, fam2, col1, 1, data));
140    memstore.add(new KeyValue(row1, fam2, col2, 1, data));
141    memstore.add(new KeyValue(row1, fam2, col3, 1, data));
142    memstore.add(new KeyValue(row1, fam2, col4, 1, data));
143    memstore.add(new KeyValue(row1, fam2, col5, 1, data));
144    memstore.add(new KeyValue(row2, fam1, col1, 1, data));
145
146    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<>(memstore.size());
147
148    KeyValue k = memstore.get(0);
149    qm.setToNewRow(k);
150
151    for (KeyValue kv : memstore) {
152      actual.add(qm.match(kv));
153    }
154
155    assertEquals(expected.size(), actual.size());
156    for (int i = 0; i < expected.size(); i++) {
157      LOG.debug("expected " + expected.get(i) + ", actual " + actual.get(i));
158      assertEquals(expected.get(i), actual.get(i));
159    }
160  }
161
162  /**
163   * Verify that {@link ScanQueryMatcher} only skips expired KeyValue instances and does not exit
164   * early from the row (skipping later non-expired KeyValues). This version mimics a Get with
165   * explicitly specified column qualifiers. n
166   */
167  @Test
168  public void testMatch_ExpiredExplicit() throws IOException {
169
170    long testTTL = 1000;
171    MatchCode[] expected = new MatchCode[] { ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
172      ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL,
173      ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
174      ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL,
175      ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW, ScanQueryMatcher.MatchCode.DONE };
176
177    long now = EnvironmentEdgeManager.currentTime();
178    UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan,
179      new ScanInfo(this.conf, fam2, 0, 1, testTTL, KeepDeletedCells.FALSE,
180        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
181      get.getFamilyMap().get(fam2), now - testTTL, now, null);
182
183    KeyValue[] kvs = new KeyValue[] { new KeyValue(row1, fam2, col1, now - 100, data),
184      new KeyValue(row1, fam2, col2, now - 50, data),
185      new KeyValue(row1, fam2, col3, now - 5000, data),
186      new KeyValue(row1, fam2, col4, now - 500, data),
187      new KeyValue(row1, fam2, col5, now - 10000, data),
188      new KeyValue(row2, fam1, col1, now - 10, data) };
189
190    KeyValue k = kvs[0];
191    qm.setToNewRow(k);
192
193    List<MatchCode> actual = new ArrayList<>(kvs.length);
194    for (KeyValue kv : kvs) {
195      actual.add(qm.match(kv));
196    }
197
198    assertEquals(expected.length, actual.size());
199    for (int i = 0; i < expected.length; i++) {
200      LOG.debug("expected " + expected[i] + ", actual " + actual.get(i));
201      assertEquals(expected[i], actual.get(i));
202    }
203  }
204
205  /**
206   * Verify that {@link ScanQueryMatcher} only skips expired KeyValue instances and does not exit
207   * early from the row (skipping later non-expired KeyValues). This version mimics a Get with
208   * wildcard-inferred column qualifiers. n
209   */
210  @Test
211  public void testMatch_ExpiredWildcard() throws IOException {
212
213    long testTTL = 1000;
214    MatchCode[] expected =
215      new MatchCode[] { ScanQueryMatcher.MatchCode.INCLUDE, ScanQueryMatcher.MatchCode.INCLUDE,
216        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL, ScanQueryMatcher.MatchCode.INCLUDE,
217        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL, ScanQueryMatcher.MatchCode.DONE };
218
219    long now = EnvironmentEdgeManager.currentTime();
220    UserScanQueryMatcher qm = UserScanQueryMatcher.create(scan, new ScanInfo(this.conf, fam2, 0, 1,
221      testTTL, KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false), null,
222      now - testTTL, now, null);
223
224    KeyValue[] kvs = new KeyValue[] { new KeyValue(row1, fam2, col1, now - 100, data),
225      new KeyValue(row1, fam2, col2, now - 50, data),
226      new KeyValue(row1, fam2, col3, now - 5000, data),
227      new KeyValue(row1, fam2, col4, now - 500, data),
228      new KeyValue(row1, fam2, col5, now - 10000, data),
229      new KeyValue(row2, fam1, col1, now - 10, data) };
230    KeyValue k = kvs[0];
231    qm.setToNewRow(k);
232
233    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<>(kvs.length);
234    for (KeyValue kv : kvs) {
235      actual.add(qm.match(kv));
236    }
237
238    assertEquals(expected.length, actual.size());
239    for (int i = 0; i < expected.length; i++) {
240      LOG.debug("expected " + expected[i] + ", actual " + actual.get(i));
241      assertEquals(expected[i], actual.get(i));
242    }
243  }
244
245  private static class AlwaysIncludeAndSeekNextRowFilter extends FilterBase {
246
247    @Override
248    public ReturnCode filterKeyValue(final Cell c) throws IOException {
249      return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
250    }
251  }
252
253  @Test
254  public void testMatchWhenFilterReturnsIncludeAndSeekNextRow() throws IOException {
255    List<MatchCode> expected = new ArrayList<>();
256    expected.add(ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW);
257    expected.add(ScanQueryMatcher.MatchCode.DONE);
258
259    Scan scanWithFilter = new Scan(scan).setFilter(new AlwaysIncludeAndSeekNextRowFilter());
260
261    long now = EnvironmentEdgeManager.currentTime();
262
263    // scan with column 2,4,5
264    UserScanQueryMatcher qm = UserScanQueryMatcher.create(
265      scanWithFilter, new ScanInfo(this.conf, fam2, 0, 1, ttl, KeepDeletedCells.FALSE,
266        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
267      get.getFamilyMap().get(fam2), now - ttl, now, null);
268
269    List<KeyValue> memstore = new ArrayList<>();
270    // ColumnTracker will return INCLUDE_AND_SEEK_NEXT_COL , and filter will return
271    // INCLUDE_AND_SEEK_NEXT_ROW, so final match code will be INCLUDE_AND_SEEK_NEXT_ROW.
272    memstore.add(new KeyValue(row1, fam2, col2, 1, data));
273    memstore.add(new KeyValue(row2, fam1, col1, data));
274
275    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<>(memstore.size());
276    KeyValue k = memstore.get(0);
277    qm.setToNewRow(k);
278
279    for (KeyValue kv : memstore) {
280      actual.add(qm.match(kv));
281    }
282
283    assertEquals(expected.size(), actual.size());
284    for (int i = 0; i < expected.size(); i++) {
285      LOG.debug("expected " + expected.get(i) + ", actual " + actual.get(i));
286      assertEquals(expected.get(i), actual.get(i));
287    }
288  }
289
290  private static class AlwaysIncludeFilter extends FilterBase {
291    @Override
292    public ReturnCode filterKeyValue(final Cell c) throws IOException {
293      return ReturnCode.INCLUDE;
294    }
295  }
296
297  /**
298   * Here is the unit test for UserScanQueryMatcher#mergeFilterResponse, when the number of cells
299   * exceed the versions requested in scan, we should return SEEK_NEXT_COL, but if current match
300   * code is INCLUDE_AND_SEEK_NEXT_ROW, we can optimize to choose the max step between SEEK_NEXT_COL
301   * and INCLUDE_AND_SEEK_NEXT_ROW, which is SEEK_NEXT_ROW. <br/>
302   */
303  @Test
304  public void testMergeFilterResponseCase1() throws IOException {
305    List<MatchCode> expected = new ArrayList<>();
306    expected.add(MatchCode.INCLUDE);
307    expected.add(MatchCode.INCLUDE);
308    expected.add(MatchCode.SEEK_NEXT_ROW);
309
310    Scan scanWithFilter = new Scan(scan).setFilter(new AlwaysIncludeFilter()).readVersions(2);
311
312    long now = EnvironmentEdgeManager.currentTime();
313    // scan with column 2,4,5, the family with maxVersion = 3
314    UserScanQueryMatcher qm = UserScanQueryMatcher.create(
315      scanWithFilter, new ScanInfo(this.conf, fam2, 0, 3, ttl, KeepDeletedCells.FALSE,
316        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
317      get.getFamilyMap().get(fam2), now - ttl, now, null);
318
319    List<KeyValue> memstore = new ArrayList<>();
320    memstore.add(new KeyValue(row1, fam1, col5, 1, data)); // match code will be INCLUDE
321    memstore.add(new KeyValue(row1, fam1, col5, 2, data)); // match code will be INCLUDE
322
323    // match code will be SEEK_NEXT_ROW , which is max(INCLUDE_AND_SEEK_NEXT_ROW, SEEK_NEXT_COL).
324    memstore.add(new KeyValue(row1, fam1, col5, 3, data));
325
326    KeyValue k = memstore.get(0);
327    qm.setToNewRow(k);
328
329    for (int i = 0; i < memstore.size(); i++) {
330      assertEquals(expected.get(i), qm.match(memstore.get(i)));
331    }
332
333    scanWithFilter = new Scan(scan).setFilter(new AlwaysIncludeFilter()).readVersions(1);
334    qm = UserScanQueryMatcher.create(
335      scanWithFilter, new ScanInfo(this.conf, fam2, 0, 2, ttl, KeepDeletedCells.FALSE,
336        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
337      get.getFamilyMap().get(fam2), now - ttl, now, null);
338
339    List<KeyValue> memstore2 = new ArrayList<>();
340    memstore2.add(new KeyValue(row2, fam1, col2, 1, data)); // match code will be INCLUDE
341    // match code will be SEEK_NEXT_COL, which is max(INCLUDE_AND_SEEK_NEXT_COL, SEEK_NEXT_COL).
342    memstore2.add(new KeyValue(row2, fam1, col2, 2, data));
343
344    k = memstore2.get(0);
345    qm.setToNewRow(k);
346
347    assertEquals(MatchCode.INCLUDE, qm.match(memstore2.get(0)));
348    assertEquals(MatchCode.SEEK_NEXT_COL, qm.match(memstore2.get(1)));
349  }
350
351  /**
352   * Here is the unit test for UserScanQueryMatcher#mergeFilterResponse: the match code may be
353   * changed to SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_COL after merging with filterResponse, even
354   * if the passed match code is neither SEEK_NEXT_COL nor INCLUDE_AND_SEEK_NEXT_COL. In that case,
355   * we need to make sure that the ColumnTracker has been switched to the next column. <br/>
356   * An effective test way is: we only need to check the cell from getKeyForNextColumn(). because
357   * that as long as the UserScanQueryMatcher returns SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_COL,
358   * UserScanQueryMatcher#getKeyForNextColumn should return an cell whose column is larger than the
359   * current cell's.
360   */
361  @Test
362  public void testMergeFilterResponseCase2() throws Exception {
363    List<MatchCode> expected = new ArrayList<>();
364    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
365    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
366    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
367    expected.add(ScanQueryMatcher.MatchCode.SEEK_NEXT_COL);
368
369    Scan scanWithFilter = new Scan(scan).setFilter(new AlwaysIncludeFilter()).readVersions(3);
370
371    long now = EnvironmentEdgeManager.currentTime();
372
373    // scan with column 2,4,5, the family with maxVersion = 5
374    UserScanQueryMatcher qm = UserScanQueryMatcher.create(
375      scanWithFilter, new ScanInfo(this.conf, fam2, 0, 5, ttl, KeepDeletedCells.FALSE,
376        HConstants.DEFAULT_BLOCKSIZE, 0, rowComparator, false),
377      get.getFamilyMap().get(fam2), now - ttl, now, null);
378
379    List<KeyValue> memstore = new ArrayList<>();
380
381    memstore.add(new KeyValue(row1, fam1, col2, 1, data)); // match code will be INCLUDE
382    memstore.add(new KeyValue(row1, fam1, col2, 2, data)); // match code will be INCLUDE
383    memstore.add(new KeyValue(row1, fam1, col2, 3, data)); // match code will be INCLUDE
384    memstore.add(new KeyValue(row1, fam1, col2, 4, data)); // match code will be SEEK_NEXT_COL
385
386    KeyValue k = memstore.get(0);
387    qm.setToNewRow(k);
388
389    for (int i = 0; i < memstore.size(); i++) {
390      assertEquals(expected.get(i), qm.match(memstore.get(i)));
391    }
392
393    // For last cell, the query matcher will return SEEK_NEXT_COL, and the
394    // ColumnTracker will skip to the next column, which is col4.
395    Cell lastCell = memstore.get(memstore.size() - 1);
396    Cell nextCell = qm.getKeyForNextColumn(lastCell);
397    assertArrayEquals(nextCell.getQualifierArray(), col4);
398  }
399}