diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java index a83908030a2..eafc9331df4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.client; +import java.util.Arrays; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; @@ -31,4 +33,46 @@ public class ClientUtil { public static Cursor createCursor(byte[] row) { return new Cursor(row); } + + /** + *

When scanning for a prefix the scan should stop immediately after the the last row that + * has the specified prefix. This method calculates the closest next rowKey immediately following + * the given rowKeyPrefix.

+ *

IMPORTANT: This converts a rowKeyPrefix into a rowKey.

+ *

If the prefix is an 'ASCII' string put into a byte[] then this is easy because you can + * simply increment the last byte of the array. + * But if your application uses real binary rowids you may run into the scenario that your + * prefix is something like:

+ *    { 0x12, 0x23, 0xFF, 0xFF }
+ * Then this stopRow needs to be fed into the actual scan
+ *    { 0x12, 0x24 } (Notice that it is shorter now)
+ * This method calculates the correct stop row value for this usecase. + * + * @param rowKeyPrefix the rowKeyPrefix. + * @return the closest next rowKey immediately following the given rowKeyPrefix. + */ + public static byte[] calculateTheClosestNextRowKeyForPrefix(byte[] rowKeyPrefix) { + // Essentially we are treating it like an 'unsigned very very long' and doing +1 manually. + // Search for the place where the trailing 0xFFs start + int offset = rowKeyPrefix.length; + while (offset > 0) { + if (rowKeyPrefix[offset - 1] != (byte) 0xFF) { + break; + } + offset--; + } + + if (offset == 0) { + // We got an 0xFFFF... (only FFs) stopRow value which is + // the last possible prefix before the end of the table. + // So set it to stop at the 'end of the table' + return HConstants.EMPTY_END_ROW; + } + + // Copy the right length of the original + byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset); + // And increment the last one + newStopRow[newStopRow.length - 1]++; + return newStopRow; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java index 541b3d8f1d0..e965062336f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -530,53 +529,11 @@ public class Scan extends Query { setStopRow(HConstants.EMPTY_END_ROW); } else { this.setStartRow(rowPrefix); - this.setStopRow(calculateTheClosestNextRowKeyForPrefix(rowPrefix)); + this.setStopRow(ClientUtil.calculateTheClosestNextRowKeyForPrefix(rowPrefix)); } return this; } - /** - *

When scanning for a prefix the scan should stop immediately after the the last row that - * has the specified prefix. This method calculates the closest next rowKey immediately following - * the given rowKeyPrefix.

- *

IMPORTANT: This converts a rowKeyPrefix into a rowKey.

- *

If the prefix is an 'ASCII' string put into a byte[] then this is easy because you can - * simply increment the last byte of the array. - * But if your application uses real binary rowids you may run into the scenario that your - * prefix is something like:

- *    { 0x12, 0x23, 0xFF, 0xFF }
- * Then this stopRow needs to be fed into the actual scan
- *    { 0x12, 0x24 } (Notice that it is shorter now)
- * This method calculates the correct stop row value for this usecase. - * - * @param rowKeyPrefix the rowKeyPrefix. - * @return the closest next rowKey immediately following the given rowKeyPrefix. - */ - private byte[] calculateTheClosestNextRowKeyForPrefix(byte[] rowKeyPrefix) { - // Essentially we are treating it like an 'unsigned very very long' and doing +1 manually. - // Search for the place where the trailing 0xFFs start - int offset = rowKeyPrefix.length; - while (offset > 0) { - if (rowKeyPrefix[offset - 1] != (byte) 0xFF) { - break; - } - offset--; - } - - if (offset == 0) { - // We got an 0xFFFF... (only FFs) stopRow value which is - // the last possible prefix before the end of the table. - // So set it to stop at the 'end of the table' - return HConstants.EMPTY_END_ROW; - } - - // Copy the right length of the original - byte[] newStopRow = Arrays.copyOfRange(rowKeyPrefix, 0, offset); - // And increment the last one - newStopRow[newStopRow.length - 1]++; - return newStopRow; - } - /** * Get all available versions. * @return this diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java index 2eebab5fdbb..8c750bc44fe 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.ClientUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.ByteStringer; @@ -77,6 +78,33 @@ public class MultiRowRangeFilter extends FilterBase { this.ranges = new RangeIteration(rangeList); } + /** + * Constructor for creating a MultiRowRangeFilter from multiple rowkey prefixes. + * + * As MultiRowRangeFilter javadoc says (See the solution 1 of the first statement), + * if you try to create a filter list that scans row keys corresponding to given prefixes (e.g., + * FilterList composed of multiple PrefixFilters), this constructor + * provides a way to avoid creating an inefficient one. + * + * @param rowKeyPrefixes the array of byte array + */ + public MultiRowRangeFilter(byte[][] rowKeyPrefixes) throws IOException { + this(createRangeListFromRowKeyPrefixes(rowKeyPrefixes)); + } + + private static List createRangeListFromRowKeyPrefixes(byte[][] rowKeyPrefixes) { + if (rowKeyPrefixes == null) { + throw new IllegalArgumentException("Invalid rowkey prefixes"); + } + + List list = new ArrayList<>(); + for (byte[] rowKeyPrefix: rowKeyPrefixes) { + byte[] stopRow = ClientUtil.calculateTheClosestNextRowKeyForPrefix(rowKeyPrefix); + list.add(new RowRange(rowKeyPrefix, true, stopRow, false)); + } + return list; + } + public List getRowRanges() { // Used by hbase-rest return this.rangeList; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java index b5252713366..7979e2000fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -69,6 +70,61 @@ public class TestMultiRowRangeFilter { TEST_UTIL.shutdownMiniCluster(); } + @Test + public void testRowKeyPrefixWithEmptyPrefix() throws IOException { + byte[] prefix = {}; + byte[][] rowKeyPrefixes = new byte[1][]; + rowKeyPrefixes[0] = prefix; + MultiRowRangeFilter filter = new MultiRowRangeFilter(rowKeyPrefixes); + List actualRanges = filter.getRowRanges(); + List expectedRanges = new ArrayList<>(); + expectedRanges.add( + new RowRange(HConstants.EMPTY_START_ROW, true, HConstants.EMPTY_END_ROW, false) + ); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testRowKeyPrefixWithLastIncrementablePrefix() throws IOException { + byte[] prefix = {(byte) 0x12, (byte) 0x23, (byte) 0xFF, (byte) 0xFE}; + byte[][] rowKeyPrefixes = new byte[1][]; + rowKeyPrefixes[0] = prefix; + MultiRowRangeFilter filter = new MultiRowRangeFilter(rowKeyPrefixes); + List actualRanges = filter.getRowRanges(); + List expectedRanges = new ArrayList<>(); + final byte[] expectedStop = {(byte) 0x12, (byte) 0x23, (byte) 0xFF, (byte) 0xFF}; + expectedRanges.add(new RowRange(prefix, true, expectedStop , false)); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testRowKeyPrefixWithoutLastIncrementablePrefix() throws IOException { + byte[] prefix = {(byte) 0x12, (byte) 0x23, (byte) 0xFF, (byte) 0xFF}; + byte[][] rowKeyPrefixes = new byte[1][]; + rowKeyPrefixes[0] = prefix; + MultiRowRangeFilter filter = new MultiRowRangeFilter(rowKeyPrefixes); + List actualRanges = filter.getRowRanges(); + List expectedRanges = new ArrayList<>(); + final byte[] expectedStop = {(byte) 0x12, (byte) 0x24}; + expectedRanges.add(new RowRange(prefix, true, expectedStop , false)); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testRowKeyPrefixWithMergablePrefix() throws IOException { + byte[] prefix1 = {(byte) 0x12, (byte) 0x23, (byte) 0xFF, (byte) 0xFE}; + byte[] prefix2 = {(byte) 0x12, (byte) 0x23, (byte) 0xFF, (byte) 0xFF}; + byte[][] rowKeyPrefixes = new byte[2][]; + rowKeyPrefixes[0] = prefix1; + rowKeyPrefixes[1] = prefix2; + MultiRowRangeFilter filter = new MultiRowRangeFilter(rowKeyPrefixes); + List actualRanges = filter.getRowRanges(); + List expectedRanges = new ArrayList<>(); + final byte[] expectedStop = {(byte) 0x12, (byte) 0x24}; + expectedRanges.add(new RowRange(prefix1, true, expectedStop , false)); + assertRangesEqual(expectedRanges, actualRanges); + } + @Test public void testRanges() throws IOException { byte[] key1Start = new byte[] {-3};