hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jason Rutherglen <jason.rutherg...@gmail.com>
Subject Re: Coprocessor Endpoints
Date Tue, 15 Mar 2011 22:49:05 GMT
Stack,

Solr currently skips obtaining the global term frequency (see
http://wiki.apache.org/solr/DistributedSearch under 'No distributed
idf').  This means there'll be one network trip per region.  What is
nice about coupling realtime search to HBase is the search results
will always be consistent.

Jason

On Tue, Mar 15, 2011 at 3:25 PM, Stack <stack@duboce.net> wrote:
> Jason:
>
> What about the double trip done in solr, es, nutch, etc. where first
> query is about term frequency in each index (region) and then second
> query is the actual search w/ the term distribution factored in?  Will
> you need to do something equivalent?
>
> St.Ack
>
>
> On Tue, Mar 15, 2011 at 2:53 PM, Jason Rutherglen
> <jason.rutherglen@gmail.com> wrote:
>> Gary,
>>
>> Thanks for your descriptive definition of how the HBase RPC works.
>>
>>> They will issue RPC calls in parallel to
>>> all of the in the range starting with the region containing "startRow" and
>>> ending with the region containing "endRow" (again using the row keys for the
>>> region lookups).
>>
>> I think I'll start with the single region RPC call.  Once that works,
>> use the 'execute in parallel' for distributed search, eg, it will take
>> care of the underlying details, then the code that needs to be written
>> will only entail merging the results.
>>
>> Thanks,
>>
>> Jason
>>
>> On Tue, Mar 15, 2011 at 10:11 AM, Gary Helmling <ghelmling@gmail.com> wrote:
>>> Hi Jason,
>>>
>>> That's basically correct.  To export your own RPC methods from a
>>> coprocessor, you:
>>>
>>> 1) Define an interface containing the RPC methods.  This interface must
>>> extend CoprocessorProtocol (which only requires you to implement
>>> getProtocolVersion())
>>>
>>> 2) Implement the defined RPC interface in your coprocessor
>>>
>>> 3) To call the RPC methods, you have 3 options:
>>>
>>> T HTable.coprocessorProxy(Class<T> protocol, byte[] row)
>>>
>>> Map<byte[],R> HTable.coprocessorExec(Class<T> protocol, byte[] startRow,
>>> byte[] endRow, Batch.Call<T,R> callable)
>>>
>>> void HTable.coprocessorExec(Class<T> protocol, byte[] start, byte[] end,
>>> Batch.Call<T,R> call, Batch.Callback<R> callback)
>>>
>>>
>>> The first returns a proxy instance of T to call methods against the
>>> coprocessor instance in the region containing the row key given by "byte[]
>>> row", as you described.  So the row key is just used to lookup the region.
>>>
>>> The last two are a bit tricker.  They will issue RPC calls in parallel to
>>> all of the in the range starting with the region containing "startRow" and
>>> ending with the region containing "endRow" (again using the row keys for the
>>> region lookups).  However, to call the RPC methods, you still need a proxy
>>> instance of T, which instead of being returned directly, you get access to
>>> by providing an implementation of Batch.Call<T,R>, where T is the protocol
>>> interface and R is the type returned from your Batch.Call implemenation's
>>> call(T instance) method.
>>>
>>> For each region in the range startRow to endRow, the HBase client will call
>>> your instance of Batch.Call.call(T proxy) passing a proxy instance connected
>>> to that region.
>>>
>>> So in TestCoprocessorEndpoint, we define a simple RPC protocol:
>>>
>>> public interface ColumnAggregationProtocol extends CoprocessorProtocol {
>>>  public long sum(byte[] family, byte[] qualifier) throws IOException;
>>> }
>>>
>>>
>>> Then we invoke it using HTable.coprocessorExec() with an anonymous
>>> Batch.Call instance like so:
>>>
>>> new Batch.Call<ColumnAggregationProtocol, Long>() {
>>>    public Long call(ColumnAggregationProtocol instance)
>>>        throws IOException {
>>>        return instance.sum(TEST_FAMILY, TEST_QUALIFIER);
>>>    }
>>> }
>>>
>>> We then get back a Map<byte[],Long> from coprocessorExec() containing the
>>> result of each call() invocation, keyed by the region name.
>>>
>>> Take a look at the Batch.forMethod() helper, which will generate a
>>> Batch.Call instance for you if you just want to invoke a single method
>>> across multiple regions with coprocessorExec().
>>>
>>>
>>> --gh
>>>
>>>
>>>
>>> On Tue, Mar 15, 2011 at 9:15 AM, Jason Rutherglen <
>>> jason.rutherglen@gmail.com> wrote:
>>>
>>>> I'm taking a look at TestCoprocessorEndpoint for example, in trying to
>>>> figure out how the Coprocessor RPC works.  I think
>>>> HTable.coprocessorProxy should be used?  Which will return an
>>>> interface that when called performs the network marshaling etc.  The
>>>> purpose of the row byte[] in coprocessorProxy is to find the correct
>>>> region to make the call to?
>>>>
>>>
>>
>

Mime
View raw message