HBASE-26967 FilterList with FuzzyRowFilter and SingleColumnValueFilter evaluated with operator MUST_PASS_ONE doesn't work as expected(#4820)
Close #4820
Co-authored-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
(cherry picked from commit 382681e2d6
)
This commit is contained in:
parent
0e645ce8c4
commit
4caa19f6a9
|
@ -85,7 +85,7 @@ public abstract class FilterBase extends Filter {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fitlers that never filter by modifying the returned List of Cells can inherit this
|
* Filters that never filter by modifying the returned List of Cells can inherit this
|
||||||
* implementation that does nothing. {@inheritDoc}
|
* implementation that does nothing. {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.filter;
|
package org.apache.hadoop.hbase.filter;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -48,13 +49,20 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesP
|
||||||
* <li>1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
|
* <li>1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
|
||||||
* position can be different from the one in provided row key</li>
|
* position can be different from the one in provided row key</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
* Example: Let's assume row key format is userId_actionId_year_month. Length of userId is fixed and
|
* Example:
|
||||||
* is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively. Let's
|
* <p/>
|
||||||
* assume that we need to fetch all users that performed certain action (encoded as "99") in Jan of
|
* Let's assume row key format is userId_actionId_year_month. Length of userId is fixed and is 4,
|
||||||
* any year. Then the pair (row key, fuzzy info) would be the following: row key = "????_99_????_01"
|
* length of actionId is 2 and year and month are 4 and 2 bytes long respectively.
|
||||||
* (one can use any value instead of "?") fuzzy info =
|
* <p/>
|
||||||
* "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" I.e. fuzzy info tells the matching
|
* Let's assume that we need to fetch all users that performed certain action (encoded as "99") in
|
||||||
* mask is "????_99_????_01", where at ? can be any value.
|
* Jan of any year. Then the pair (row key, fuzzy info) would be the following:
|
||||||
|
*
|
||||||
|
* <pre>
|
||||||
|
* row key = "????_99_????_01" (one can use any value instead of "?")
|
||||||
|
* fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00"
|
||||||
|
* </pre>
|
||||||
|
*
|
||||||
|
* I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public class FuzzyRowFilter extends FilterBase {
|
public class FuzzyRowFilter extends FilterBase {
|
||||||
|
@ -71,6 +79,15 @@ public class FuzzyRowFilter extends FilterBase {
|
||||||
|
|
||||||
private final byte processedWildcardMask;
|
private final byte processedWildcardMask;
|
||||||
private List<Pair<byte[], byte[]>> fuzzyKeysData;
|
private List<Pair<byte[], byte[]>> fuzzyKeysData;
|
||||||
|
// Used to record whether we want to skip the current row.
|
||||||
|
// Usually we should use filterRowKey here but in the current scan implementation, if filterRowKey
|
||||||
|
// returns true, we will just skip to next row, instead of calling getNextCellHint to determine
|
||||||
|
// the actual next row, so we need to implement filterCell and return SEEK_NEXT_USING_HINT to let
|
||||||
|
// upper layer call getNextCellHint.
|
||||||
|
// And if we do not implement filterRow, sometimes we will get incorrect result when using
|
||||||
|
// FuzzyRowFilter together with other filters, please see the description for HBASE-26967 for more
|
||||||
|
// details.
|
||||||
|
private boolean filterRow;
|
||||||
private boolean done = false;
|
private boolean done = false;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -172,6 +189,16 @@ public class FuzzyRowFilter extends FilterBase {
|
||||||
return filterCell(c);
|
return filterCell(c);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reset() throws IOException {
|
||||||
|
filterRow = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean filterRow() throws IOException {
|
||||||
|
return filterRow;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReturnCode filterCell(final Cell c) {
|
public ReturnCode filterCell(final Cell c) {
|
||||||
final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0;
|
final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0;
|
||||||
|
@ -189,7 +216,7 @@ public class FuzzyRowFilter extends FilterBase {
|
||||||
}
|
}
|
||||||
// NOT FOUND -> seek next using hint
|
// NOT FOUND -> seek next using hint
|
||||||
lastFoundIndex = -1;
|
lastFoundIndex = -1;
|
||||||
|
filterRow = true;
|
||||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.hadoop.hbase.filter;
|
package org.apache.hadoop.hbase.filter;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
@ -27,6 +28,7 @@ import java.util.List;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
|
import org.apache.hadoop.hbase.CompareOperator;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
@ -370,7 +372,6 @@ public class TestFuzzyRowFilterEndToEnd {
|
||||||
assertEquals(expectedSize, found);
|
assertEquals(expectedSize, found);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
|
||||||
@Test
|
@Test
|
||||||
public void testFilterList() throws Exception {
|
public void testFilterList() throws Exception {
|
||||||
String cf = "f";
|
String cf = "f";
|
||||||
|
@ -413,7 +414,6 @@ public class TestFuzzyRowFilterEndToEnd {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
private void runTest(Table hTable, int expectedSize) throws IOException {
|
private void runTest(Table hTable, int expectedSize) throws IOException {
|
||||||
// [0, 2, ?, ?, ?, ?, 0, 0, 0, 1]
|
// [0, 2, ?, ?, ?, ?, 0, 0, 0, 1]
|
||||||
byte[] fuzzyKey1 = new byte[10];
|
byte[] fuzzyKey1 = new byte[10];
|
||||||
|
@ -471,4 +471,57 @@ public class TestFuzzyRowFilterEndToEnd {
|
||||||
|
|
||||||
assertEquals(expectedSize, results.size());
|
assertEquals(expectedSize, results.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testHBASE26967() throws IOException {
|
||||||
|
byte[] row1 = Bytes.toBytes("1");
|
||||||
|
byte[] row2 = Bytes.toBytes("2");
|
||||||
|
String cf1 = "f1";
|
||||||
|
String cf2 = "f2";
|
||||||
|
String cq1 = "col1";
|
||||||
|
String cq2 = "col2";
|
||||||
|
|
||||||
|
Table ht =
|
||||||
|
TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), new String[] { cf1, cf2 });
|
||||||
|
|
||||||
|
// Put data
|
||||||
|
List<Put> puts = Lists.newArrayList();
|
||||||
|
puts.add(new Put(row1).addColumn(Bytes.toBytes(cf1), Bytes.toBytes(cq1), Bytes.toBytes("a1")));
|
||||||
|
puts.add(new Put(row1).addColumn(Bytes.toBytes(cf2), Bytes.toBytes(cq2), Bytes.toBytes("a2")));
|
||||||
|
puts.add(new Put(row2).addColumn(Bytes.toBytes(cf1), Bytes.toBytes(cq1), Bytes.toBytes("b1")));
|
||||||
|
puts.add(new Put(row2).addColumn(Bytes.toBytes(cf2), Bytes.toBytes(cq2), Bytes.toBytes("b2")));
|
||||||
|
ht.put(puts);
|
||||||
|
|
||||||
|
TEST_UTIL.flush();
|
||||||
|
|
||||||
|
// FuzzyRowFilter
|
||||||
|
List<Pair<byte[], byte[]>> data = Lists.newArrayList();
|
||||||
|
byte[] fuzzyKey = Bytes.toBytes("1");
|
||||||
|
byte[] mask = new byte[] { 0 };
|
||||||
|
data.add(new Pair<>(fuzzyKey, mask));
|
||||||
|
FuzzyRowFilter fuzzyRowFilter = new FuzzyRowFilter(data);
|
||||||
|
|
||||||
|
// SingleColumnValueFilter
|
||||||
|
Filter singleColumnValueFilter = new SingleColumnValueFilter(Bytes.toBytes(cf2),
|
||||||
|
Bytes.toBytes(cq2), CompareOperator.EQUAL, Bytes.toBytes("x"));
|
||||||
|
|
||||||
|
// FilterList
|
||||||
|
FilterList filterList = new FilterList(Operator.MUST_PASS_ONE);
|
||||||
|
filterList.addFilter(Lists.newArrayList(fuzzyRowFilter, singleColumnValueFilter));
|
||||||
|
|
||||||
|
Scan scan = new Scan();
|
||||||
|
scan.setFilter(filterList);
|
||||||
|
|
||||||
|
ResultScanner scanner = ht.getScanner(scan);
|
||||||
|
Result rs = scanner.next();
|
||||||
|
assertEquals(0, Bytes.compareTo(row1, rs.getRow()));
|
||||||
|
|
||||||
|
// The two cells (1,f1,col1,a1) (1,f2,col2,a2)
|
||||||
|
assertEquals(2, rs.listCells().size());
|
||||||
|
|
||||||
|
// Only one row who's rowKey=1
|
||||||
|
assertNull(scanner.next());
|
||||||
|
|
||||||
|
TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue