From 98d8bb71fd3812b43d8fed73cb4dfb1b999af7e0 Mon Sep 17 00:00:00 2001 From: Guanghao Zhang Date: Tue, 19 Feb 2019 12:09:47 +0800 Subject: [PATCH] HBASE-21922 BloomContext#sanityCheck may failed when use ROWPREFIX_DELIMITED bloom filter --- .../org/apache/hadoop/hbase/util/BloomContext.java | 10 -- .../regionserver/TestRowPrefixBloomFilter.java | 132 ++++++++++++--------- 2 files changed, 73 insertions(+), 69 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java index 31394e8..5097139 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java @@ -51,20 +51,10 @@ public abstract class BloomContext { public void writeBloom(Cell cell) throws IOException { // only add to the bloom filter on a new, unique key if (isNewKey(cell)) { - sanityCheck(cell); bloomFilterWriter.append(cell); } } - private void sanityCheck(Cell cell) throws IOException { - if (this.getLastCell() != null) { - if (comparator.compare(cell, this.getLastCell()) <= 0) { - throw new IOException("Added a key not lexically larger than" + " previous. Current cell = " - + cell + ", prevCell = " + this.getLastCell()); - } - } - } - /** * Adds the last bloom key to the HFile Writer as part of StorefileWriter close. * @param writer diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java index 647f450..f947540 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java @@ -82,6 +82,14 @@ public class TestRowPrefixBloomFilter { private static final String invalidFormatter = "%08d"; private static final String prefixFormatter = "%010d"; private static final String suffixFormatter = "%010d"; + private static final int prefixRowCount = 50; + private static final int suffixRowCount = 5; + private static final int infixCount = suffixRowCount; + private static final int fixedLengthExpKeys = prefixRowCount; + private static final int delimitedExpKeys = + prefixRowCount / 2 * suffixRowCount + prefixRowCount / 2; + private static final BloomType[] bts = + { BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED }; @Rule public TestName name = new TestName(); @@ -132,8 +140,7 @@ public class TestRowPrefixBloomFilter { return reader.getStoreFileScanner(false, false, false, 0, 0, false); } - private void writeStoreFile(final Path f, BloomType bt, int expKeys, int prefixRowCount, - int suffixRowCount) throws IOException { + private void writeStoreFile(final Path f, BloomType bt, int expKeys) throws IOException { HFileContext meta = new HFileContextBuilder() .withBlockSize(BLOCKSIZE_SMALL) .withChecksumType(CKTYPE) @@ -152,18 +159,20 @@ public class TestRowPrefixBloomFilter { for (int i = 0; i < prefixRowCount; i += 2) { // prefix rows String prefixRow = String.format(prefixFormatter, i); for (int j = 0; j < suffixRowCount; j++) { // suffix rows - String row = prefixRow + "#" + String.format(suffixFormatter, j); - KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), - Bytes.toBytes("col"), now, Bytes.toBytes("value")); + String row = generateRowWithSuffix(bt, prefixRow, j, j); + KeyValue kv = + new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("col"), now, + Bytes.toBytes("value")); writer.append(kv); } } //Put with invalid row style - for (int i = prefixRowCount; i < prefixRowCount*2; i += 2) { // prefix rows + for (int i = prefixRowCount; i < prefixRowCount * 2; i += 2) { // prefix rows String row = String.format(invalidFormatter, i); - KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), - Bytes.toBytes("col"), now, Bytes.toBytes("value")); + KeyValue kv = + new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("col"), now, + Bytes.toBytes("value")); writer.append(kv); } } finally { @@ -171,18 +180,28 @@ public class TestRowPrefixBloomFilter { } } + private String generateRowWithSuffix(BloomType bloomType, String prefixRow, int infix, int suffix) { + StringBuilder row = new StringBuilder(prefixRow); + if (bloomType == BloomType.ROWPREFIX_DELIMITED) { + // Generate different length row for BloomType.ROWPREFIX_DELIMITED + for (int i = infixCount; i > infix; i--) { + row.append("!"); + } + } + row.append(delimiter); + row.append(String.format(suffixFormatter, suffix)); + return row.toString(); + } + @Test public void testRowPrefixBloomFilter() throws Exception { FileSystem fs = FileSystem.getLocal(conf); - BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED}; - int prefixRowCount = 50; - int suffixRowCount = 10; - int expKeys = 50; - float expErr = 2*prefixRowCount*suffixRowCount*err; - for (int x : new int[]{0,1}) { + float expErr = 2 * prefixRowCount * suffixRowCount * err; + for (BloomType bt : bts) { + int expKeys = bt == BloomType.ROWPREFIX_FIXED_LENGTH ? fixedLengthExpKeys : delimitedExpKeys; // write the file Path f = new Path(testDir, name.getMethodName()); - writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount); + writeStoreFile(f, bt, expKeys); // read the file StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true, @@ -191,11 +210,11 @@ public class TestRowPrefixBloomFilter { reader.loadBloomfilter(); //check basic param - assertEquals(bt[x], reader.getBloomFilterType()); - if (bt[x] == BloomType.ROWPREFIX_FIXED_LENGTH) { + assertEquals(bt, reader.getBloomFilterType()); + if (bt == BloomType.ROWPREFIX_FIXED_LENGTH) { assertEquals(prefixLength, reader.getPrefixLength()); assertEquals("null", Bytes.toStringBinary(reader.getDelimiter())); - } else if (bt[x] == BloomType.ROWPREFIX_DELIMITED){ + } else if (bt == BloomType.ROWPREFIX_DELIMITED){ assertEquals(-1, reader.getPrefixLength()); assertEquals(delimiter, Bytes.toStringBinary(reader.getDelimiter())); } @@ -210,18 +229,20 @@ public class TestRowPrefixBloomFilter { for (int i = 0; i < prefixRowCount; i++) { // prefix rows String prefixRow = String.format(prefixFormatter, i); for (int j = 0; j < suffixRowCount; j++) { // suffix rows - String startRow = prefixRow + "#" + String.format(suffixFormatter, j); - String stopRow = prefixRow + "#" + String.format(suffixFormatter, j+1); + String startRow = generateRowWithSuffix(bt, prefixRow, j, j); + String stopRow = generateRowWithSuffix(bt, prefixRow, j, j + 1); Scan scan = new Scan().withStartRow(Bytes.toBytes(startRow)) .withStopRow(Bytes.toBytes(stopRow)); boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); boolean shouldPrefixRowExist = i % 2 == 0; if (shouldPrefixRowExist) { if (!exists) { + LOG.debug("++ falseNeg startRow is " + startRow + " and stop row is " + stopRow); falseNeg++; } } else { if (exists) { + LOG.debug("++ startRow is " + startRow + " and stop row is " + stopRow); falsePos++; } } @@ -247,6 +268,7 @@ public class TestRowPrefixBloomFilter { fs.delete(f, true); assertEquals("False negatives: " + falseNeg, 0, falseNeg); int maxFalsePos = (int) (2 * expErr); + LOG.debug("false pos is " + falsePos); assertTrue("Too many false positives: " + falsePos + " (err=" + err + ", expected no more than " + maxFalsePos + ")", falsePos <= maxFalsePos); @@ -256,14 +278,11 @@ public class TestRowPrefixBloomFilter { @Test public void testRowPrefixBloomFilterWithGet() throws Exception { FileSystem fs = FileSystem.getLocal(conf); - BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED}; - int prefixRowCount = 50; - int suffixRowCount = 10; - int expKeys = 50; - for (int x : new int[]{0,1}) { + for (BloomType bt : bts) { + int expKeys = bt == BloomType.ROWPREFIX_FIXED_LENGTH ? fixedLengthExpKeys : delimitedExpKeys; // write the file Path f = new Path(testDir, name.getMethodName()); - writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount); + writeStoreFile(f, bt, expKeys); StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf); @@ -278,14 +297,14 @@ public class TestRowPrefixBloomFilter { //Get with valid row style //prefix row in bloom String prefixRow = String.format(prefixFormatter, prefixRowCount-2); - String row = prefixRow + "#" + String.format(suffixFormatter, 0); + String row = generateRowWithSuffix(bt, prefixRow, 0, 0); Scan scan = new Scan(new Get(Bytes.toBytes(row))); boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertTrue(exists); // prefix row not in bloom prefixRow = String.format(prefixFormatter, prefixRowCount-1); - row = prefixRow + "#" + String.format(suffixFormatter, 0); + row = generateRowWithSuffix(bt, prefixRow, 0, 0); scan = new Scan(new Get(Bytes.toBytes(row))); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertFalse(exists); @@ -294,13 +313,13 @@ public class TestRowPrefixBloomFilter { // ROWPREFIX: the length of row is less than prefixLength // ROWPREFIX_DELIMITED: Row does not contain delimiter // row in bloom - row = String.format(invalidFormatter, prefixRowCount+2); + row = String.format(invalidFormatter, prefixRowCount + 2); scan = new Scan(new Get(Bytes.toBytes(row))); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertTrue(exists); // row not in bloom - row = String.format(invalidFormatter, prefixRowCount+1); + row = String.format(invalidFormatter, prefixRowCount + 1); scan = new Scan(new Get(Bytes.toBytes(row))); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertFalse(exists); @@ -313,14 +332,11 @@ public class TestRowPrefixBloomFilter { @Test public void testRowPrefixBloomFilterWithScan() throws Exception { FileSystem fs = FileSystem.getLocal(conf); - BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED}; - int prefixRowCount = 50; - int suffixRowCount = 10; - int expKeys = 50; - for (int x : new int[]{0,1}) { + for (BloomType bt : bts) { + int expKeys = bt == BloomType.ROWPREFIX_FIXED_LENGTH ? fixedLengthExpKeys : delimitedExpKeys; // write the file Path f = new Path(testDir, name.getMethodName()); - writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount); + writeStoreFile(f, bt, expKeys); StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf); @@ -335,47 +351,45 @@ public class TestRowPrefixBloomFilter { //Scan with valid row style. startRow and stopRow have a common prefix. //And the length of the common prefix is no less than prefixLength. //prefix row in bloom - String prefixRow = String.format(prefixFormatter, prefixRowCount-2); - String startRow = prefixRow + "#" + String.format(suffixFormatter, 0); - String stopRow = prefixRow + "#" + String.format(suffixFormatter, 1); - Scan scan = new Scan().withStartRow(Bytes.toBytes(startRow)) - .withStopRow(Bytes.toBytes(stopRow)); + String prefixRow = String.format(prefixFormatter, prefixRowCount - 2); + String startRow = generateRowWithSuffix(bt, prefixRow, 0, 0); + String stopRow = generateRowWithSuffix(bt, prefixRow, 0, 1); + Scan scan = + new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow)); boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertTrue(exists); // prefix row not in bloom - prefixRow = String.format(prefixFormatter, prefixRowCount-1); - startRow = prefixRow + "#" + String.format(suffixFormatter, 0); - stopRow = prefixRow + "#" + String.format(suffixFormatter, 1); - scan = new Scan().withStartRow(Bytes.toBytes(startRow)) - .withStopRow(Bytes.toBytes(stopRow)); + prefixRow = String.format(prefixFormatter, prefixRowCount - 1); + startRow = generateRowWithSuffix(bt, prefixRow, 0, 0); + stopRow = generateRowWithSuffix(bt, prefixRow, 0, 1); + scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow)); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertFalse(exists); // There is no common prefix between startRow and stopRow. - prefixRow = String.format(prefixFormatter, prefixRowCount-2); - startRow = prefixRow + "#" + String.format(suffixFormatter, 0); + prefixRow = String.format(prefixFormatter, prefixRowCount - 2); + startRow = generateRowWithSuffix(bt, prefixRow, 0, 0); scan = new Scan().withStartRow(Bytes.toBytes(startRow)); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertTrue(exists); - if (bt[x] == BloomType.ROWPREFIX_FIXED_LENGTH) { + if (bt == BloomType.ROWPREFIX_FIXED_LENGTH) { // startRow and stopRow have a common prefix. // But the length of the common prefix is less than prefixLength. - String prefixStartRow = String.format(prefixFormatter, prefixRowCount-2); - String prefixStopRow = String.format(prefixFormatter, prefixRowCount-1); - startRow = prefixStartRow + "#" + String.format(suffixFormatter, 0); - stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0); - scan = new Scan().withStartRow(Bytes.toBytes(startRow)) - .withStopRow(Bytes.toBytes(stopRow)); + String prefixStartRow = String.format(prefixFormatter, prefixRowCount - 2); + String prefixStopRow = String.format(prefixFormatter, prefixRowCount - 1); + startRow = generateRowWithSuffix(bt, prefixStartRow, 0, 0); + stopRow = generateRowWithSuffix(bt, prefixStopRow, 0, 0); + scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow)); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); assertTrue(exists); - }else if (bt[x] == BloomType.ROWPREFIX_DELIMITED) { + } else if (bt == BloomType.ROWPREFIX_DELIMITED) { // startRow does not contain delimiter String prefixStartRow = String.format(prefixFormatter, prefixRowCount-2); String prefixStopRow = String.format(prefixFormatter, prefixRowCount-2); startRow = prefixStartRow + String.format(suffixFormatter, 0); - stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0); + stopRow = generateRowWithSuffix(bt, prefixStopRow, 0, 0); scan = new Scan().withStartRow(Bytes.toBytes(startRow)) .withStopRow(Bytes.toBytes(stopRow)); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); @@ -384,8 +398,8 @@ public class TestRowPrefixBloomFilter { // startRow contains delimiter, but stopRow does not have the same prefix as startRow. prefixStartRow = String.format(prefixFormatter, prefixRowCount-2); prefixStopRow = String.format(prefixFormatter, prefixRowCount-1); - startRow = prefixStartRow + "#" + String.format(suffixFormatter, 0); - stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0); + startRow = generateRowWithSuffix(bt, prefixStartRow, 0, 0); + stopRow = generateRowWithSuffix(bt, prefixStopRow, 0, 0); scan = new Scan().withStartRow(Bytes.toBytes(startRow)) .withStopRow(Bytes.toBytes(stopRow)); exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); -- 2.7.4