HBASE-5266 Add documentation for ColumnRangeFilter

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1239399 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2012-02-01 23:27:43 +00:00
parent 8a6d0fbca1
commit be722211fd
1 changed files with 81 additions and 2 deletions

View File

@ -1548,11 +1548,90 @@ scan.setFilter(filter);
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.html">ColumnPrefixFilter</link> can be used
to filter based on the lead portion of Column (aka Qualifier) names.
</para>
<para>A ColumnPrefixFilter seeks ahead to the first column matching the prefix in each row and for each involved column family. It can be used to efficiently
get a subset of the columns in very wide rows.
</para>
<para>Note: The same column qualifier can be used in different column families. This filter returns all matching columns.
</para>
<para>Example: Find all columns in a row and family that start with "abc"
<programlisting>
HTableInterface t = ...;
byte[] row = ...;
byte[] family = ...;
byte[] prefix = Bytes.toBytes("abc");
Scan scan = new Scan(row, row); // (optional) limit to one row
scan.addFamily(family); // (optional) limit to one family
Filter f = new ColumnPrefixFilter(prefix);
scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
}
}
rs.close();
</programlisting>
</para>
</section>
<section xml:id="client.filter.kvm.mcpf"><title>MultipleColumnPrefixFilter</title>
<para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.html">MultipleColumnPrefixFilter</link> behaves like ColumnPrefixFilter
but allows specifying multiple prefixes.
</para>
<para>Like ColumnPrefixFilter, MultipleColumnPrefixFilter efficiently seeks ahead to the first column matching the lowest prefix and also seeks past ranges of columns between prefixes.
It can be used to efficiently get discontinuous sets of columns from very wide rows.
</para>
<para>Example: Find all columns in a row and family that start with "abc" or "xyz"
<programlisting>
HTableInterface t = ...;
byte[] row = ...;
byte[] family = ...;
byte[][] prefixes = new byte[][] {Bytes.toBytes("abc"), Bytes.toBytes("xyz")};
Scan scan = new Scan(row, row); // (optional) limit to one row
scan.addFamily(family); // (optional) limit to one family
Filter f = new MultipleColumnPrefixFilter(prefixes);
scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
}
}
rs.close();
</programlisting>
</para>
</section>
<section xml:id="client.filter.kvm.crf "><title>ColumnRangeFilter</title>
<para>Use <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnRangeFilter.html">ColumnRangeFilter</link> to get a column 'slice':
i.e. if you have a million columns in a row but you only want to look at columns bbbb-bbbd.
<para>A <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/ColumnRangeFilter.html">ColumnRangeFilter</link> allows efficient intra row scanning.
</para>
<para>A ColumnRangeFilter can seek ahead to the first matching column for each involved column family. It can be used to efficiently
get a 'slice' of the columns of a very wide row.
i.e. you have a million columns in a row but you only want to look at columns bbbb-bbdd.
</para>
<para>Note: The same column qualifier can be used in different column families. This filter returns all matching columns.
</para>
<para>Example: Find all columns in a row and family between "bbbb" (inclusive) and "bbdd" (inclusive)
<programlisting>
HTableInterface t = ...;
byte[] row = ...;
byte[] family = ...;
byte[] startColumn = Bytes.toBytes("bbbb");
byte[] endColumn = Bytes.toBytes("bbdd");
Scan scan = new Scan(row, row); // (optional) limit to one row
scan.addFamily(family); // (optional) limit to one family
Filter f = new ColumnRangeFilter(startColumn, true, endColumn, true);
scan.setFilter(f);
scan.setBatch(10); // set this if there could be many columns returned
ResultScanner rs = t.getScanner(scan);
for (Result r = rs.next(); r != null; r = rs.next()) {
for (KeyValue kv : r.raw()) {
// each kv represents a column
}
}
rs.close();
</programlisting>
</para>
<para>Note: Introduced in HBase 0.92</para>
</section>
</section>