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; 019 020import static org.junit.jupiter.api.Assertions.assertEquals; 021import static org.junit.jupiter.api.Assertions.assertTrue; 022 023import java.io.IOException; 024import java.util.ArrayList; 025import java.util.Arrays; 026import java.util.Collections; 027import java.util.Comparator; 028import java.util.List; 029import java.util.NavigableSet; 030import java.util.TreeSet; 031import java.util.stream.Stream; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.Cell; 036import org.apache.hadoop.hbase.CellUtil; 037import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate; 038import org.apache.hadoop.hbase.HBaseTestingUtil; 039import org.apache.hadoop.hbase.KeyValue; 040import org.apache.hadoop.hbase.KeyValueTestUtil; 041import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 042import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 043import org.apache.hadoop.hbase.client.Put; 044import org.apache.hadoop.hbase.client.Scan; 045import org.apache.hadoop.hbase.io.compress.Compression; 046import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter; 047import org.apache.hadoop.hbase.testclassification.RegionServerTests; 048import org.apache.hadoop.hbase.testclassification.SmallTests; 049import org.apache.hadoop.hbase.util.BloomFilterUtil; 050import org.apache.hadoop.hbase.util.Bytes; 051import org.junit.jupiter.api.BeforeEach; 052import org.junit.jupiter.api.Tag; 053import org.junit.jupiter.api.TestTemplate; 054import org.junit.jupiter.params.provider.Arguments; 055import org.slf4j.Logger; 056import org.slf4j.LoggerFactory; 057 058/** 059 * Test a multi-column scanner when there is a Bloom filter false-positive. This is needed for the 060 * multi-column Bloom filter optimization. 061 */ 062@Tag(RegionServerTests.TAG) 063@Tag(SmallTests.TAG) 064@HBaseParameterizedTestTemplate(name = "{index}: bloomType={0}") 065public class TestScanWithBloomError { 066 067 private static final Logger LOG = LoggerFactory.getLogger(TestScanWithBloomError.class); 068 069 private static final String TABLE_NAME = "ScanWithBloomError"; 070 private static final String FAMILY = "myCF"; 071 private static final byte[] FAMILY_BYTES = Bytes.toBytes(FAMILY); 072 private static final String ROW = "theRow"; 073 private static final String QUALIFIER_PREFIX = "qual"; 074 private static final byte[] ROW_BYTES = Bytes.toBytes(ROW); 075 private static NavigableSet<Integer> allColIds = new TreeSet<>(); 076 private HRegion region; 077 private BloomType bloomType; 078 private FileSystem fs; 079 private Configuration conf; 080 081 private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 082 083 public static final Stream<Arguments> parameters() { 084 List<Arguments> configurations = new ArrayList<>(); 085 for (BloomType bloomType : BloomType.values()) { 086 configurations.add(Arguments.of(bloomType)); 087 } 088 return configurations.stream(); 089 } 090 091 public TestScanWithBloomError(BloomType bloomType) { 092 this.bloomType = bloomType; 093 } 094 095 @BeforeEach 096 public void setUp() throws IOException { 097 conf = TEST_UTIL.getConfiguration(); 098 fs = FileSystem.get(conf); 099 conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10); 100 } 101 102 @TestTemplate 103 public void testThreeStoreFiles() throws IOException { 104 ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder 105 .newBuilder(Bytes.toBytes(FAMILY)).setCompressionType(Compression.Algorithm.GZ) 106 .setBloomFilterType(bloomType).setMaxVersions(TestMultiColumnScanner.MAX_VERSIONS).build(); 107 region = TEST_UTIL.createTestRegion(TABLE_NAME, columnFamilyDescriptor); 108 createStoreFile(new int[] { 1, 2, 6 }); 109 createStoreFile(new int[] { 1, 2, 3, 7 }); 110 createStoreFile(new int[] { 1, 9 }); 111 scanColSet(new int[] { 1, 4, 6, 7 }, new int[] { 1, 6, 7 }); 112 113 HBaseTestingUtil.closeRegionAndWAL(region); 114 } 115 116 private void scanColSet(int[] colSet, int[] expectedResultCols) throws IOException { 117 LOG.info("Scanning column set: " + Arrays.toString(colSet)); 118 Scan scan = new Scan().withStartRow(ROW_BYTES).withStopRow(ROW_BYTES, true); 119 addColumnSetToScan(scan, colSet); 120 RegionScannerImpl scanner = region.getScanner(scan); 121 KeyValueHeap storeHeap = scanner.storeHeap; 122 assertEquals(0, storeHeap.getHeap().size()); 123 StoreScanner storeScanner = (StoreScanner) storeHeap.getCurrentForTesting(); 124 @SuppressWarnings({ "unchecked", "rawtypes" }) 125 List<StoreFileScanner> scanners = 126 (List<StoreFileScanner>) (List) storeScanner.getAllScannersForTesting(); 127 128 // Sort scanners by their HFile's modification time. 129 Collections.sort(scanners, new Comparator<StoreFileScanner>() { 130 @Override 131 public int compare(StoreFileScanner s1, StoreFileScanner s2) { 132 Path p1 = s1.getReader().getHFileReader().getPath(); 133 Path p2 = s2.getReader().getHFileReader().getPath(); 134 long t1, t2; 135 try { 136 t1 = fs.getFileStatus(p1).getModificationTime(); 137 t2 = fs.getFileStatus(p2).getModificationTime(); 138 } catch (IOException ex) { 139 throw new RuntimeException(ex); 140 } 141 return t1 < t2 ? -1 : t1 == t2 ? 1 : 0; 142 } 143 }); 144 145 StoreFileReader lastStoreFileReader = null; 146 for (StoreFileScanner sfScanner : scanners) 147 lastStoreFileReader = sfScanner.getReader(); 148 149 new HFilePrettyPrinter(conf).run( 150 new String[] { "-m", "-p", "-f", lastStoreFileReader.getHFileReader().getPath().toString() }); 151 152 // Disable Bloom filter for the last store file. The disabled Bloom filter 153 // will always return "true". 154 LOG.info("Disabling Bloom filter for: " + lastStoreFileReader.getHFileReader().getName()); 155 lastStoreFileReader.disableBloomFilterForTesting(); 156 157 List<Cell> allResults = new ArrayList<>(); 158 159 { // Limit the scope of results. 160 List<Cell> results = new ArrayList<>(); 161 while (scanner.next(results) || results.size() > 0) { 162 allResults.addAll(results); 163 results.clear(); 164 } 165 } 166 167 List<Integer> actualIds = new ArrayList<>(); 168 for (Cell kv : allResults) { 169 String qual = Bytes.toString(CellUtil.cloneQualifier(kv)); 170 assertTrue(qual.startsWith(QUALIFIER_PREFIX)); 171 actualIds.add(Integer.valueOf(qual.substring(QUALIFIER_PREFIX.length()))); 172 } 173 List<Integer> expectedIds = new ArrayList<>(); 174 for (int expectedId : expectedResultCols) 175 expectedIds.add(expectedId); 176 177 LOG.info("Column ids returned: " + actualIds + ", expected: " + expectedIds); 178 assertEquals(expectedIds.toString(), actualIds.toString()); 179 } 180 181 private void addColumnSetToScan(Scan scan, int[] colIds) { 182 for (int colId : colIds) { 183 scan.addColumn(FAMILY_BYTES, Bytes.toBytes(qualFromId(colId))); 184 } 185 } 186 187 private String qualFromId(int colId) { 188 return QUALIFIER_PREFIX + colId; 189 } 190 191 private void createStoreFile(int[] colIds) throws IOException { 192 Put p = new Put(ROW_BYTES); 193 for (int colId : colIds) { 194 long ts = Long.MAX_VALUE; 195 String qual = qualFromId(colId); 196 allColIds.add(colId); 197 KeyValue kv = KeyValueTestUtil.create(ROW, FAMILY, qual, ts, 198 TestMultiColumnScanner.createValue(ROW, qual, ts)); 199 p.add(kv); 200 } 201 region.put(p); 202 region.flush(true); 203 } 204 205}