1   /*
2    * Copyright 2011 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  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Arrays;
25  import java.util.Collection;
26  import java.util.Collections;
27  import java.util.Comparator;
28  import java.util.List;
29  import java.util.NavigableSet;
30  import java.util.TreeSet;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.HBaseTestingUtility;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.KeyValueTestUtil;
40  import org.apache.hadoop.hbase.client.Put;
41  import org.apache.hadoop.hbase.client.Scan;
42  import org.apache.hadoop.hbase.io.hfile.Compression;
43  import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter;
44  import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.junit.Before;
47  import org.junit.Test;
48  import org.junit.runner.RunWith;
49  import org.junit.runners.Parameterized;
50  import org.junit.runners.Parameterized.Parameters;
51  
52  import static org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner.*;
53  import static org.junit.Assert.*;
54  
55  /**
56   * Test a multi-column scanner when there is a Bloom filter false-positive.
57   * This is needed for the multi-column Bloom filter optimization.
58   */
59  @RunWith(Parameterized.class)
60  public class TestScanWithBloomError {
61  
62    private static final Log LOG =
63      LogFactory.getLog(TestScanWithBloomError.class);
64  
65    private static final String TABLE_NAME = "ScanWithBloomError";
66    private static final String ROW = "theRow";
67    private static final String QUALIFIER_PREFIX = "qual";
68    private static final byte[] ROW_BYTES = Bytes.toBytes(ROW);
69    private static NavigableSet<Integer> allColIds = new TreeSet<Integer>();
70    private HRegion region;
71    private StoreFile.BloomType bloomType;
72    private FileSystem fs;
73    private Configuration conf;
74  
75    private final static HBaseTestingUtility TEST_UTIL =
76      new HBaseTestingUtility();
77  
78    @Parameters
79    public static final Collection<Object[]> parameters() {
80      List<Object[]> configurations = new ArrayList<Object[]>();
81      for (StoreFile.BloomType bloomType : StoreFile.BloomType.values()) {
82        configurations.add(new Object[] { bloomType });
83      }
84      return configurations;
85    }
86  
87    public TestScanWithBloomError(StoreFile.BloomType bloomType) {
88      this.bloomType = bloomType;
89    }
90  
91    @Before
92    public void setUp() throws IOException{
93      conf = TEST_UTIL.getConfiguration();
94      fs = FileSystem.get(conf);
95    }
96  
97    @Test
98    public void testThreeStoreFiles() throws IOException {
99      region = createRegion(TABLE_NAME, Compression.Algorithm.GZ, bloomType);
100     createStoreFile(new int[] {1, 2, 6});
101     createStoreFile(new int[] {1, 2, 3, 7});
102     createStoreFile(new int[] {1, 9});
103     scanColSet(new int[]{1, 4, 6, 7}, new int[]{1, 6, 7});
104 
105     region.close();
106   }
107 
108   private void scanColSet(int[] colSet, int[] expectedResultCols)
109       throws IOException {
110     LOG.info("Scanning column set: " + Arrays.toString(colSet));
111     Scan scan = new Scan(ROW_BYTES, ROW_BYTES);
112     addColumnSetToScan(scan, colSet);
113     RegionScannerImpl scanner = (RegionScannerImpl) region.getScanner(scan);
114     KeyValueHeap storeHeap = scanner.getStoreHeapForTesting();
115     assertEquals(0, storeHeap.getHeap().size());
116     StoreScanner storeScanner =
117         (StoreScanner) storeHeap.getCurrentForTesting();
118     @SuppressWarnings({ "unchecked", "rawtypes" })
119     List<StoreFileScanner> scanners = (List<StoreFileScanner>)
120         (List) storeScanner.getAllScannersForTesting();
121 
122     // Sort scanners by their HFile's modification time.
123     Collections.sort(scanners, new Comparator<StoreFileScanner>() {
124       @Override
125       public int compare(StoreFileScanner s1, StoreFileScanner s2) {
126         Path p1 = s1.getReaderForTesting().getHFileReader().getPath();
127         Path p2 = s2.getReaderForTesting().getHFileReader().getPath();
128         long t1, t2;
129         try {
130           t1 = fs.getFileStatus(p1).getModificationTime();
131           t2 = fs.getFileStatus(p2).getModificationTime();
132         } catch (IOException ex) {
133           throw new RuntimeException(ex);
134         }
135         return t1 < t2 ? -1 : t1 == t2 ? 1 : 0;
136       }
137     });
138 
139     StoreFile.Reader lastStoreFileReader = null;
140     for (StoreFileScanner sfScanner : scanners)
141       lastStoreFileReader = sfScanner.getReaderForTesting();
142 
143     new HFilePrettyPrinter().run(new String[]{ "-m", "-p", "-f",
144         lastStoreFileReader.getHFileReader().getPath().toString()});
145 
146     // Disable Bloom filter for the last store file. The disabled Bloom filter
147     // will always return "true".
148     LOG.info("Disabling Bloom filter for: "
149         + lastStoreFileReader.getHFileReader().getName());
150     lastStoreFileReader.disableBloomFilterForTesting();
151 
152     List<KeyValue> allResults = new ArrayList<KeyValue>();
153 
154     { // Limit the scope of results.
155       List<KeyValue> results = new ArrayList<KeyValue>();
156       while (scanner.next(results) || results.size() > 0) {
157         allResults.addAll(results);
158         results.clear();
159       }
160     }
161 
162     List<Integer> actualIds = new ArrayList<Integer>();
163     for (KeyValue kv : allResults) {
164       String qual = Bytes.toString(kv.getQualifier());
165       assertTrue(qual.startsWith(QUALIFIER_PREFIX));
166       actualIds.add(Integer.valueOf(qual.substring(
167           QUALIFIER_PREFIX.length())));
168     }
169     List<Integer> expectedIds = new ArrayList<Integer>();
170     for (int expectedId : expectedResultCols)
171       expectedIds.add(expectedId);
172 
173     LOG.info("Column ids returned: " + actualIds + ", expected: "
174         + expectedIds);
175     assertEquals(expectedIds.toString(), actualIds.toString());
176   }
177 
178   private void addColumnSetToScan(Scan scan, int[] colIds) {
179     for (int colId : colIds)
180       scan.addColumn(FAMILY_BYTES, Bytes.toBytes(qualFromId(colId)));
181   }
182 
183   private String qualFromId(int colId) {
184     return QUALIFIER_PREFIX + colId;
185   }
186 
187   private void createStoreFile(int[] colIds)
188       throws IOException {
189     Put p = new Put(ROW_BYTES);
190     for (int colId : colIds) {
191       long ts = Long.MAX_VALUE;
192       String qual = qualFromId(colId);
193       allColIds.add(colId);
194       KeyValue kv = KeyValueTestUtil.create(ROW, FAMILY,
195           qual, ts, createValue(ROW, qual, ts));
196       p.add(kv);
197     }
198     region.put(p);
199     region.flushcache();
200   }
201 
202 }