blob: 1c1279de8811d0a4feb329f4a97984e131d59dd6 [file] [log] [blame]
////
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
////
[[thrift]]
= Thrift API and Filter Language
:doctype: book
:numbered:
:toc: left
:icons: font
:experimental:
Apache link:https://thrift.apache.org/[Thrift] is a cross-platform, cross-language development framework.
HBase includes a Thrift API and filter language.
The Thrift API relies on client and server processes.
You can configure Thrift for secure authentication at the server and client side, by following the procedures in <<security.client.thrift>> and <<security.gateway.thrift>>.
The rest of this chapter discusses the filter language provided by the Thrift API.
[[thrift.filter_language]]
== Filter Language
Thrift Filter Language was introduced in HBase 0.92.
It allows you to perform server-side filtering when accessing HBase over Thrift or in the HBase shell.
You can find out more about shell integration by using the `scan help` command in the shell.
You specify a filter as a string, which is parsed on the server to construct the filter.
[[general_syntax]]
=== General Filter String Syntax
A simple filter expression is expressed as a string:
----
“FilterName (argument, argument,... , argument)”
----
Keep the following syntax guidelines in mind.
* Specify the name of the filter followed by the comma-separated argument list in parentheses.
* If the argument represents a string, it should be enclosed in single quotes (`'`).
* Arguments which represent a boolean, an integer, or a comparison operator (such as <, >, or !=), should not be enclosed in quotes
* The filter name must be a single word.
All ASCII characters are allowed except for whitespace, single quotes and parentheses.
* The filter's arguments can contain any ASCII character.
If single quotes are present in the argument, they must be escaped by an additional preceding single quote.
=== Compound Filters and Operators
.Binary Operators
`AND`::
If the `AND` operator is used, the key-value must satisfy both filters.
`OR`::
If the `OR` operator is used, the key-value must satisfy at least one of the filters.
.Unary Operators
`SKIP`::
For a particular row, if any of the key-values fail the filter condition, the entire row is skipped.
`WHILE`::
For a particular row, key-values will be emitted until a key-value is reached that fails the filter condition.
.Compound Operators
====
You can combine multiple operators to create a hierarchy of filters, such as the following example:
[source]
----
(Filter1 AND Filter2) OR (Filter3 AND Filter4)
----
====
=== Order of Evaluation
. Parentheses have the highest precedence.
. The unary operators `SKIP` and `WHILE` are next, and have the same precedence.
. The binary operators follow. `AND` has highest precedence, followed by `OR`.
.Precedence Example
====
[source]
----
Filter1 AND Filter2 OR Filter
is evaluated as
(Filter1 AND Filter2) OR Filter3
----
[source]
----
Filter1 AND SKIP Filter2 OR Filter3
is evaluated as
(Filter1 AND (SKIP Filter2)) OR Filter3
----
====
You can use parentheses to explicitly control the order of evaluation.
=== Compare Operator
The following compare operators are provided:
. LESS (<)
. LESS_OR_EQUAL (<=)
. EQUAL (=)
. NOT_EQUAL (!=)
. GREATER_OR_EQUAL (>=)
. GREATER (>)
. NO_OP (no operation)
The client should use the symbols (<, <=, =, !=, >, >=) to express compare operators.
=== Comparator
A comparator can be any of the following:
. _BinaryComparator_ - This lexicographically compares against the specified byte array using Bytes.compareTo(byte[], byte[])
. _BinaryPrefixComparator_ - This lexicographically compares against a specified byte array.
It only compares up to the length of this byte array.
. _RegexStringComparator_ - This compares against the specified byte array using the given regular expression.
Only EQUAL and NOT_EQUAL comparisons are valid with this comparator
. _SubStringComparator_ - This tests if the given substring appears in a specified byte array.
The comparison is case insensitive.
Only EQUAL and NOT_EQUAL comparisons are valid with this comparator
The general syntax of a comparator is: `ComparatorType:ComparatorValue`
The ComparatorType for the various comparators is as follows:
. _BinaryComparator_ - binary
. _BinaryPrefixComparator_ - binaryprefix
. _RegexStringComparator_ - regexstring
. _SubStringComparator_ - substring
The ComparatorValue can be any value.
.Example ComparatorValues
. `binary:abc` will match everything that is lexicographically greater than "abc"
. `binaryprefix:abc` will match everything whose first 3 characters are lexicographically equal to "abc"
. `regexstring:ab*yz` will match everything that doesn't begin with "ab" and ends with "yz"
. `substring:abc123` will match everything that begins with the substring "abc123"
[[examplephpclientprogram]]
=== Example PHP Client Program that uses the Filter Language
[source,php]
----
<?
$_SERVER['PHP_ROOT'] = realpath(dirname(__FILE__).'/..');
require_once $_SERVER['PHP_ROOT'].'/flib/__flib.php';
flib_init(FLIB_CONTEXT_SCRIPT);
require_module('storage/hbase');
$hbase = new HBase('<server_name_running_thrift_server>', <port on which thrift server is running>);
$hbase->open();
$client = $hbase->getClient();
$result = $client->scannerOpenWithFilterString('table_name', "(PrefixFilter ('row2') AND (QualifierFilter (>=, 'binary:xyz'))) AND (TimestampsFilter ( 123, 456))");
$to_print = $client->scannerGetList($result,1);
while ($to_print) {
print_r($to_print);
$to_print = $client->scannerGetList($result,1);
}
$client->scannerClose($result);
?>
----
=== Example Filter Strings
* `"PrefixFilter ('Row') AND PageFilter (1) AND FirstKeyOnlyFilter ()"` will return all key-value pairs that match the following conditions:
+
. The row containing the key-value should have prefix _Row_
. The key-value must be located in the first row of the table
. The key-value pair must be the first key-value in the row
+
* `"(RowFilter (=, 'binary:Row 1') AND TimeStampsFilter (74689, 89734)) OR ColumnRangeFilter ('abc', true, 'xyz', false))"` will return all key-value pairs that match both the following conditions:
** The key-value is in a row having row key _Row 1_
** The key-value must have a timestamp of either 74689 or 89734.
** Or it must match the following condition:
*** The key-value pair must be in a column that is lexicographically >= abc and < xyz 
+
* `"SKIP ValueFilter (0)"` will skip the entire row if any of the values in the row is not 0
[[individualfiltersyntax]]
=== Individual Filter Syntax
KeyOnlyFilter::
This filter doesn't take any arguments.
It returns only the key component of each key-value.
FirstKeyOnlyFilter::
This filter doesn't take any arguments.
It returns only the first key-value from each row.
PrefixFilter::
This filter takes one argument – a prefix of a row key.
It returns only those key-values present in a row that starts with the specified row prefix
ColumnPrefixFilter::
This filter takes one argument – a column prefix.
It returns only those key-values present in a column that starts with the specified column prefix.
The column prefix must be of the form: `“qualifier”`.
MultipleColumnPrefixFilter::
This filter takes a list of column prefixes.
It returns key-values that are present in a column that starts with any of the specified column prefixes.
Each of the column prefixes must be of the form: `“qualifier”`.
ColumnCountGetFilter::
This filter takes one argument – a limit.
It returns the first limit number of columns in the table.
PageFilter::
This filter takes one argument – a page size.
It returns page size number of rows from the table.
ColumnPaginationFilter::
This filter takes two arguments – a limit and offset.
It returns limit number of columns after offset number of columns.
It does this for all the rows.
InclusiveStopFilter::
This filter takes one argument – a row key on which to stop scanning.
It returns all key-values present in rows up to and including the specified row.
TimeStampsFilter::
This filter takes a list of timestamps.
It returns those key-values whose timestamps matches any of the specified timestamps.
RowFilter::
This filter takes a compare operator and a comparator.
It compares each row key with the comparator using the compare operator and if the comparison returns true, it returns all the key-values in that row.
Family Filter::
This filter takes a compare operator and a comparator.
It compares each column family name with the comparator using the compare operator and if the comparison returns true, it returns all the Cells in that column family.
QualifierFilter::
This filter takes a compare operator and a comparator.
It compares each qualifier name with the comparator using the compare operator and if the comparison returns true, it returns all the key-values in that column.
ValueFilter::
This filter takes a compare operator and a comparator.
It compares each value with the comparator using the compare operator and if the comparison returns true, it returns that key-value.
DependentColumnFilter::
This filter takes two arguments – a family and a qualifier.
It tries to locate this column in each row and returns all key-values in that row that have the same timestamp.
If the row doesn't contain the specified column – none of the key-values in that row will be returned.
SingleColumnValueFilter::
This filter takes a column family, a qualifier, a compare operator and a comparator.
If the specified column is not found – all the columns of that row will be emitted.
If the column is found and the comparison with the comparator returns true, all the columns of the row will be emitted.
If the condition fails, the row will not be emitted.
SingleColumnValueExcludeFilter::
This filter takes the same arguments and behaves same as SingleColumnValueFilter – however, if the column is found and the condition passes, all the columns of the row will be emitted except for the tested column value.
ColumnRangeFilter::
This filter is used for selecting only those keys with columns that are between minColumn and maxColumn.
It also takes two boolean variables to indicate whether to include the minColumn and maxColumn or not.