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 2a144d266bea..ab0513260c6b 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 @@ -182,6 +182,33 @@ private boolean isPreprocessedMask(byte[] mask) { return true; } + /** + * Returns the Fuzzy keys in the format expected by the constructor. + * @return the Fuzzy keys in the format expected by the constructor + */ + public List> getFuzzyKeys() { + List> returnList = new ArrayList<>(fuzzyKeysData.size()); + for (Pair fuzzyKey : fuzzyKeysData) { + Pair returnKey = new Pair<>(); + // This won't revert the original key's don't care values, but we don't care. + returnKey.setFirst(Arrays.copyOf(fuzzyKey.getFirst(), fuzzyKey.getFirst().length)); + byte[] returnMask = Arrays.copyOf(fuzzyKey.getSecond(), fuzzyKey.getSecond().length); + if (UNSAFE_UNALIGNED && isPreprocessedMask(returnMask)) { + // Revert the preprocessing. + for (int i = 0; i < returnMask.length; i++) { + if (returnMask[i] == -1) { + returnMask[i] = 0; // -1 >> 0 + } else if (returnMask[i] == processedWildcardMask) { + returnMask[i] = 1; // 0 or 2 >> 1 depending on mask version + } + } + } + returnKey.setSecond(returnMask); + returnList.add(returnKey); + } + return returnList; + } + @Deprecated @Override public ReturnCode filterKeyValue(final Cell c) { diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java index 591ac44c1b89..d6ef7a570e56 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.filter.FuzzyRowFilter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; import org.apache.hadoop.hbase.filter.MultiRowRangeFilter; @@ -75,6 +76,7 @@ import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider; @@ -277,6 +279,63 @@ public boolean equals(Object obj) { } + static class FuzzyKeyModel { + + protected byte[] key; + + protected byte[] mask; + + public FuzzyKeyModel() { + } + + public FuzzyKeyModel(Pair keyWithMask) { + this.key = keyWithMask.getFirst(); + this.mask = keyWithMask.getSecond(); + } + + public Pair build() { + return new Pair<>(key, mask); + } + + public byte[] getKey() { + return key; + } + + public void setKey(byte[] key) { + this.key = key; + } + + public byte[] getMask() { + return mask; + } + + public void setMask(byte[] mask) { + this.mask = mask; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(key); + result = prime * result + Arrays.hashCode(mask); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof FuzzyKeyModel)) { + return false; + } + FuzzyKeyModel other = (FuzzyKeyModel) obj; + return Arrays.equals(key, other.key) && Arrays.equals(mask, other.mask); + } + + } + // A grab bag of fields, would have been a union if this were C. // These are null by default and will only be serialized if set (non null). @XmlAttribute @@ -319,6 +378,8 @@ public boolean equals(Object obj) { private List ranges; @XmlElement public List timestamps; + @XmlElement + private List fuzzyKeys; static enum FilterType { ColumnCountGetFilter, @@ -343,7 +404,8 @@ static enum FilterType { SkipFilter, TimestampsFilter, ValueFilter, - WhileMatchFilter + WhileMatchFilter, + FuzzyRowFilter } public FilterModel() { @@ -456,6 +518,12 @@ public FilterModel(Filter filter) { this.filters = new ArrayList<>(); this.filters.add(new FilterModel(((WhileMatchFilter) filter).getFilter())); break; + case FuzzyRowFilter: + this.fuzzyKeys = new ArrayList<>(((FuzzyRowFilter) filter).getFuzzyKeys().size()); + for (Pair keyWithMask : ((FuzzyRowFilter) filter).getFuzzyKeys()) { + this.fuzzyKeys.add(new FuzzyKeyModel(keyWithMask)); + } + break; default: throw new RuntimeException("unhandled filter type " + type); } @@ -567,6 +635,14 @@ public Filter build() { case WhileMatchFilter: filter = new WhileMatchFilter(filters.get(0).build()); break; + case FuzzyRowFilter: { + ArrayList> fuzzyKeyArgs = new ArrayList<>(fuzzyKeys.size()); + for (FuzzyKeyModel keyModel : fuzzyKeys) { + fuzzyKeyArgs.add(keyModel.build()); + } + filter = new FuzzyRowFilter(fuzzyKeyArgs); + } + break; default: throw new RuntimeException("unhandled filter type: " + type); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java index 0d01fc888c51..88e685ec0778 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.filter.FuzzyRowFilter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.MultiRowRangeFilter; import org.apache.hadoop.hbase.filter.PageFilter; @@ -70,6 +71,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -229,7 +231,7 @@ private static void verifyScan(Scan s, long expectedRows, long expectedKeys) thr int rows = cells.getRows().size(); assertEquals( - "Scanned too many rows! Only expected " + expectedRows + " total but scanned " + rows, + "Unexpected number of rows! Expected " + expectedRows + " total but scanned " + rows, expectedRows, rows); for (RowModel row : cells.getRows()) { int count = row.getCells().size(); @@ -972,4 +974,25 @@ public void testMultiRowRangeFilter() throws Exception { s.setFilter(new MultiRowRangeFilter(ranges)); verifyScan(s, expectedRows, expectedKeys); } + + @Test + public void testFuzzyRowFilter() throws Exception { + long expectedRows = 4; + long expectedKeys = colsPerRow; + List> fuzzyKeys = new ArrayList<>(); + + // Exact match for ROWS_ONE[0] (one row) + byte[] rowOneMask = new byte[ROWS_ONE[0].length]; + Arrays.fill(rowOneMask, (byte) 0); + fuzzyKeys.add(new Pair<>(ROWS_ONE[0], rowOneMask)); + // All ROW_TWO keys (three rows) + byte[] rowTwoMask = new byte[ROWS_TWO[0].length]; + Arrays.fill(rowTwoMask, (byte) 0); + rowTwoMask[rowTwoMask.length - 1] = (byte) 1; + fuzzyKeys.add(new Pair<>(ROWS_TWO[2], rowTwoMask)); + + Scan s = new Scan(); + s.setFilter(new FuzzyRowFilter(fuzzyKeys)); + verifyScan(s, expectedRows, expectedKeys); + } }