From 16fe69a9fff79d88ddf831afacd151399361e80c Mon Sep 17 00:00:00 2001 From: honma Date: Thu, 20 Aug 2015 20:46:01 +0800 Subject: [PATCH] HBASE-14269 FuzzyRowFilter omits certain rows when multiple fuzzy key exist --- .../apache/hadoop/hbase/filter/FuzzyRowFilter.java | 135 ++++++++++----------- .../hbase/filter/TestFuzzyRowFilterEndToEnd.java | 78 ++++++++---- 2 files changed, 121 insertions(+), 92 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index 661400b..c8c5ef7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.PriorityQueue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -81,8 +81,8 @@ public class FuzzyRowFilter extends FilterBase { p = fuzzyKeysData.get(i); if (p.getFirst().length != p.getSecond().length) { Pair readable = - new Pair(Bytes.toStringBinary(p.getFirst()), Bytes.toStringBinary(p - .getSecond())); + new Pair(Bytes.toStringBinary(p.getFirst()), Bytes.toStringBinary(p + .getSecond())); throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable); } // update mask ( 0 -> -1 (0xff), 1 -> 0) @@ -145,8 +145,8 @@ public class FuzzyRowFilter extends FilterBase { final int index = i % size; Pair fuzzyData = fuzzyKeysData.get(index); SatisfiesCode satisfiesCode = - satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(), - fuzzyData.getFirst(), fuzzyData.getSecond()); + satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(), + fuzzyData.getFirst(), fuzzyData.getSecond()); if (satisfiesCode == SatisfiesCode.YES) { lastFoundIndex = index; return ReturnCode.INCLUDE; @@ -160,82 +160,81 @@ public class FuzzyRowFilter extends FilterBase { @Override public Cell getNextCellHint(Cell currentCell) { - boolean result = true; - if (tracker.needsUpdate()) { - result = tracker.updateTracker(currentCell); - } + boolean result = tracker.updateTracker(currentCell); if (result == false) { done = true; return null; } byte[] nextRowKey = tracker.nextRow(); - // We need to compare nextRowKey with currentCell - int compareResult = CellComparator.COMPARATOR.compareRows(currentCell, nextRowKey, 0, - nextRowKey.length); - if ((reversed && compareResult < 0) || (!reversed && compareResult > 0)) { - // This can happen when we have multilpe filters and some other filter - // returns next row with hint which is larger (smaller for reverse) - // than the current (really?) - result = tracker.updateTracker(currentCell); - if (result == false) { - done = true; - return null; - } else { - nextRowKey = tracker.nextRow(); - } - } return KeyValueUtil.createFirstOnRow(nextRowKey); } /** - * If we have multiple fuzzy keys, row tracker should improve overall performance It calculates - * all next rows (one per every fuzzy key), sort them accordingly (ascending for regular and - * descending for reverse). Next time getNextCellHint is called we check row tracker first and - * return next row from the tracker if it exists, if there are no rows in the tracker we update - * tracker with a current cell and return first row. + * If we have multiple fuzzy keys, row tracker should improve overall performance. It calculates + * all next rows (one per every fuzzy key) and put them (the fuzzy key is bundled) into a priority + * queue so that the smallest row key always appears at queue head, which helps to decide the "Next + * Cell Hint". As scanning going on, the number of candidate rows in the RowTracker will remain to be + * the size of fuzzy keys until some of the fuzzy keys won't possibly have matches any more. */ private class RowTracker { - private final List nextRows; - private int next = -1; + private final PriorityQueue>> nextRows; + private boolean initialized = false; RowTracker() { - nextRows = new ArrayList(); - } - - boolean needsUpdate() { - return next == -1 || next == nextRows.size(); + nextRows = + new PriorityQueue>>(fuzzyKeysData.size(), + new Comparator>>() { + @Override + public int compare(Pair> o1, + Pair> o2) { + int compare = Bytes.compareTo(o1.getFirst(), o2.getFirst()); + if (!isReversed()) { + return compare; + } else { + return -compare; + } + } + }); } byte[] nextRow() { - if (next < 0 || next == nextRows.size()) return null; - return nextRows.get(next++); + if (nextRows.isEmpty()) { + throw new IllegalStateException( + "NextRows should not be empty, make sure to call it after updateTracker return true"); + } else { + return nextRows.peek().getFirst(); + } } boolean updateTracker(Cell currentCell) { - nextRows.clear(); - for (Pair fuzzyData : fuzzyKeysData) { - byte[] nextRowKeyCandidate = - getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), - currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(), - fuzzyData.getSecond()); - if (nextRowKeyCandidate == null) { - continue; + if (!initialized) { + for (Pair fuzzyData : fuzzyKeysData) { + updateWith(currentCell, fuzzyData); } - nextRows.add(nextRowKeyCandidate); - } - // Sort all next row candidates - Collections.sort(nextRows, new Comparator() { - @Override - public int compare(byte[] o1, byte[] o2) { - if (reversed) { - return -Bytes.compareTo(o1, o2); - } else { - return Bytes.compareTo(o1, o2); - } + initialized = true; + } else { + while (!nextRows.isEmpty() && !lessThan(currentCell, nextRows.peek().getFirst())) { + Pair> head = nextRows.poll(); + Pair fuzzyData = head.getSecond(); + updateWith(currentCell, fuzzyData); } - }); - next = 0; - return nextRows.size() > 0; + } + return !nextRows.isEmpty(); + } + + boolean lessThan(Cell currentCell, byte[] nextRowKey) { + int compareResult = + CellComparator.COMPARATOR.compareRows(currentCell, nextRowKey, 0, nextRowKey.length); + return (!isReversed() && compareResult < 0) || (isReversed() && compareResult > 0); + } + + void updateWith(Cell currentCell, Pair fuzzyData) { + byte[] nextRowKeyCandidate = + getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), currentCell.getRowOffset(), + currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); + if (nextRowKeyCandidate != null) { + nextRows.add(new Pair>(nextRowKeyCandidate, fuzzyData)); + } } } @@ -314,12 +313,12 @@ public class FuzzyRowFilter extends FilterBase { @VisibleForTesting static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, - byte[] fuzzyKeyMeta) { + byte[] fuzzyKeyMeta) { return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); } static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { if (UnsafeAccess.isAvailable() == false) { return satisfiesNoUnsafe(reverse, row, offset, length, fuzzyKeyBytes, fuzzyKeyMeta); @@ -382,8 +381,8 @@ public class FuzzyRowFilter extends FilterBase { return SatisfiesCode.YES; } - static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset, - int length, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset, int length, + byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { if (row == null) { // do nothing, let scan to proceed return SatisfiesCode.YES; @@ -435,7 +434,7 @@ public class FuzzyRowFilter extends FilterBase { @VisibleForTesting static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, - byte[] fuzzyKeyMeta) { + byte[] fuzzyKeyMeta) { return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); } @@ -512,7 +511,7 @@ public class FuzzyRowFilter extends FilterBase { */ @VisibleForTesting static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { // To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than // the given one we do the following: // 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes @@ -522,7 +521,7 @@ public class FuzzyRowFilter extends FilterBase { // It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position // values than otherwise. byte[] result = - Arrays.copyOf(fuzzyKeyBytes, length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length); + Arrays.copyOf(fuzzyKeyBytes, length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length); if (reverse && length > fuzzyKeyBytes.length) { // we need trailing 0xff's instead of trailing 0x00's for (int i = fuzzyKeyBytes.length; i < result.length; i++) { @@ -589,7 +588,7 @@ public class FuzzyRowFilter extends FilterBase { Pair thisData = this.fuzzyKeysData.get(i); Pair otherData = other.fuzzyKeysData.get(i); if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst()) && Bytes.equals( - thisData.getSecond(), otherData.getSecond()))) { + thisData.getSecond(), otherData.getSecond()))) { return false; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java index 018a9a0..416f1ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java @@ -62,10 +62,11 @@ public class TestFuzzyRowFilterEndToEnd { private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final Log LOG = LogFactory.getLog(TestFuzzyRowFilterEndToEnd.class); - private static int firstPartCardinality = 50; - private static int secondPartCardinality = 40; + private static int firstPartCardinality = 3; + private static int secondPartCardinality = 50; + private static int thirdPartCardinality = 40; private static int colQualifiersTotal = 50; - private static int totalFuzzyKeys = secondPartCardinality / 2; + private static int totalFuzzyKeys = thirdPartCardinality / 2; private static String table = "TestFuzzyRowFilterEndToEnd"; @@ -119,25 +120,28 @@ public class TestFuzzyRowFilterEndToEnd { // 4 byte qualifier // 4 byte value - for (int i1 = 0; i1 < firstPartCardinality; i1++) { - if ((i1 % 1000) == 0) LOG.info("put " + i1); + for (int i0 = 0; i0 < firstPartCardinality; i0++) { - for (int i2 = 0; i2 < secondPartCardinality; i2++) { - byte[] rk = new byte[10]; + for (int i1 = 0; i1 < secondPartCardinality; i1++) { + if ((i1 % 1000) == 0) LOG.info("put " + i1); - ByteBuffer buf = ByteBuffer.wrap(rk); - buf.clear(); - buf.putShort((short) 2); - buf.putInt(i1); - buf.putInt(i2); - for (int c = 0; c < colQualifiersTotal; c++) { - byte[] cq = new byte[4]; - Bytes.putBytes(cq, 0, Bytes.toBytes(c), 0, 4); + for (int i2 = 0; i2 < thirdPartCardinality; i2++) { + byte[] rk = new byte[10]; - Put p = new Put(rk); - p.setDurability(Durability.SKIP_WAL); - p.add(cf.getBytes(), cq, Bytes.toBytes(c)); - ht.put(p); + ByteBuffer buf = ByteBuffer.wrap(rk); + buf.clear(); + buf.putShort((short) i0); + buf.putInt(i1); + buf.putInt(i2); + for (int c = 0; c < colQualifiersTotal; c++) { + byte[] cq = new byte[4]; + Bytes.putBytes(cq, 0, Bytes.toBytes(c), 0, 4); + + Put p = new Put(rk); + p.setDurability(Durability.SKIP_WAL); + p.add(cf.getBytes(), cq, Bytes.toBytes(c)); + ht.put(p); + } } } } @@ -145,11 +149,12 @@ public class TestFuzzyRowFilterEndToEnd { TEST_UTIL.flush(); // test passes - runTest(ht); + runTest1(ht); + runTest2(ht); } - private void runTest(Table hTable) throws IOException { + private void runTest1(Table hTable) throws IOException { // [0, 2, ?, ?, ?, ?, 0, 0, 0, 1] byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 }; @@ -169,7 +174,31 @@ public class TestFuzzyRowFilterEndToEnd { list.add(pair); } - int expectedSize = firstPartCardinality * totalFuzzyKeys * colQualifiersTotal; + int expectedSize = secondPartCardinality * totalFuzzyKeys * colQualifiersTotal; + FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list); + // Filters are not stateless - we can't reuse them + FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list); + + // regular test + runScanner(hTable, expectedSize, fuzzyRowFilter0); + // optimized from block cache + runScanner(hTable, expectedSize, fuzzyRowFilter1); + + } + + private void runTest2(Table hTable) throws IOException { + // [0, 0, ?, ?, ?, ?, 0, 0, 0, 0] or [0, 2, ?, ?, ?, ?, 0, 0, 0, 2] + + byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 }; + byte ff = (byte) 63; + + List> list = new ArrayList>(); + + list.add(new Pair(new byte[] { 0, 0, ff, ff, ff, ff, 0, 0, 0, 0 }, mask)); + list.add(new Pair(new byte[] { 0, 2, ff, ff, ff, ff, 0, 0, 0, 2 }, mask)); + + int expectedSize = 2 * secondPartCardinality * colQualifiersTotal; + FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list); // Filters are not stateless - we can't reuse them FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list); @@ -208,7 +237,7 @@ public class TestFuzzyRowFilterEndToEnd { assertEquals(expectedSize, found); } - + @SuppressWarnings("deprecation") @Test public void testFilterList() throws Exception { @@ -284,7 +313,8 @@ public class TestFuzzyRowFilterEndToEnd { runScanner(hTable, expectedSize, fuzzyRowFilter1, fuzzyRowFilter2); } - private void runScanner(Table hTable, int expectedSize, Filter filter1, Filter filter2) throws IOException { + private void runScanner(Table hTable, int expectedSize, Filter filter1, Filter filter2) + throws IOException { String cf = "f"; Scan scan = new Scan(); scan.addFamily(cf.getBytes()); -- 1.8.5.2 (Apple Git-48)