Return-Path: X-Original-To: apmail-hbase-user-archive@www.apache.org Delivered-To: apmail-hbase-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 46CA3101FD for ; Wed, 2 Oct 2013 21:42:16 +0000 (UTC) Received: (qmail 85173 invoked by uid 500); 2 Oct 2013 21:42:14 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 85076 invoked by uid 500); 2 Oct 2013 21:42:13 -0000 Mailing-List: contact user-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hbase.apache.org Delivered-To: mailing list user@hbase.apache.org Received: (qmail 85068 invoked by uid 99); 2 Oct 2013 21:42:13 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Oct 2013 21:42:13 +0000 X-ASF-Spam-Status: No, hits=2.8 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS,URI_HEX X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of ghelmling@gmail.com designates 209.85.216.52 as permitted sender) Received: from [209.85.216.52] (HELO mail-qa0-f52.google.com) (209.85.216.52) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Oct 2013 21:42:08 +0000 Received: by mail-qa0-f52.google.com with SMTP id k4so1052780qaq.18 for ; Wed, 02 Oct 2013 14:41:48 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=WQmpbKcrGfclIPpUwF+U/c1bjdda6Hb0sTHfMBtMp2U=; b=WL2x7YL0eTcy/p7IvKVyYB1n8FqY00j419ncTpHRtFYm38BTMaPA37StYHfBBgcPPr 5ohmBP9RnWSxm2QVHLqper7CjZPARO2EsSbXIQ49FbAvYODo1+twGonN6HCf0d6e2kmM R5E3JmtGwWNDOnceut6TU2Ap+gl3kb3vLAp/b5iaIpcAgREb5lZc37zaJIIeisBaBYYT O1Tey+l4NoRaSgwJ324ysyHYBjfNjjORhvDbgAkhp7L/1T8zDTYQyCf0GS7+IAJSc8kI a9Qwi/howERwGJ7QMSP/uS5RJwtU3K12lRKiX/gNEvFBdBDioPqGZqUP9ynMopk2Y+oc gOLQ== MIME-Version: 1.0 X-Received: by 10.229.101.136 with SMTP id c8mr5674415qco.17.1380750107933; Wed, 02 Oct 2013 14:41:47 -0700 (PDT) Received: by 10.229.80.69 with HTTP; Wed, 2 Oct 2013 14:41:47 -0700 (PDT) In-Reply-To: <1380723681032-4051395.post@n3.nabble.com> References: <1380657313645-4051383.post@n3.nabble.com> <1380723681032-4051395.post@n3.nabble.com> Date: Wed, 2 Oct 2013 14:41:47 -0700 Message-ID: Subject: Re: Endpoint and Observer work together? From: Gary Helmling To: user@hbase.apache.org Content-Type: multipart/alternative; boundary=001a11c3faceb5104504e7c8ef52 X-Virus-Checked: Checked by ClamAV on apache.org --001a11c3faceb5104504e7c8ef52 Content-Type: text/plain; charset=ISO-8859-1 Your DemoObserver is not being invoked because DemoEndpoint is opening a scanner directly on the region: RegionCoprocessorEnvironment env =(RegionCoprocessorEnvironment)getEnvironment(); InternalScanner scanner = env.getRegion().getScanner(scan); The RegionObserver.postScannerNext() hook is invoked higher up in the client call stack. If the processing of these two coprocessors is so tightly related, then I'd recommend just combining them to a single class (a RegionObserver can also be an endpoint): public class DemoObserver extends BaseRegionObserver implements DemoProtocol { Or if for some reason this is difficult to do, then separate out your KeyValue handling into a shared class that can be use by both DemoObserver.postScannerNext() and the InternalScanner result handling in DemoEndpoint.scanRows(). On Wed, Oct 2, 2013 at 7:21 AM, rgaimari wrote: > Hi, > > I've created some demo code to show the problem. Here's the Observer: > > ... > public class DemoObserver extends BaseRegionObserver { > byte[] personFamily = Bytes.toBytes("Person"); > > @Override > public boolean postScannerNext( > ObserverContext e, InternalScanner s, > List results, int limit, boolean hasMore) > throws IOException { > List newResults = new ArrayList(); > for (Result result : results) { > List newKVList = new ArrayList(); > for (KeyValue kv : result.list()) { > String newVal = Bytes.toString(kv.getValue()).toUpperCase(); > newKVList.add(new KeyValue(kv.getRow(), kv.getFamily(), > kv.getQualifier(), kv.getTimestamp(), > Bytes.toBytes(newVal))); > } > newResults.add(new Result(newKVList)); > } > results.clear(); > results.addAll(newResults); > return super.postScannerNext(e, s, results, limit, hasMore); > } > } > > > And here's the Endpoint: > > ... > public class DemoEndpoint extends BaseEndpointCoprocessor implements > DemoProtocol { > > @Override > public List scanRows(Filter filter) throws IOException { > Scan scan = new Scan(); > scan.setFilter(filter); > RegionCoprocessorEnvironment env = > (RegionCoprocessorEnvironment)getEnvironment(); > InternalScanner scanner = env.getRegion().getScanner(scan); > > List retValues = new ArrayList(); > boolean more = false; > List res = new ArrayList(); > do { > res.clear(); > more = scanner.next(res); > if (res != null) > retValues.addAll(res); > } while (more); > > scanner.close(); > return retValues; > } > } > > They are loaded in separate jar files, and they are both attached to the > table: > > 1.9.3p448 :009 > describe 'Demo' > DESCRIPTION ENABLED > {NAME => 'Demo', coprocessor$2 => 'hdfs:///user/hduser/DemoEndpoi true > nt.jar|demo.DemoEndpoint|1001', coprocessor$1 => 'hdfs:///user/hd > user/DemoObserver.jar|demo.DemoObserver|1', FAMILIES => [{NAME => > 'Person', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER => 'NONE', > REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION => 'NONE', > MIN_VERSIONS => '0', TTL => '2147483647', KEEP_DELETED_CELLS => > 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false', ENCODE_ON_DI > SK => 'true', BLOCKCACHE => 'true'}]} > 1 row(s) in 0.0880 seconds > > If I run a test where I do a scan directly on the client (with no filter), > I > get the following results: > > , ,
, <123 MAIN STREET> > , , , > , ,
, <234 ELM STREET> > , , , > , ,
, <345 SCOTT STREET> > , , , > > The values are all capitalized, as the Observer was supposed to do. > However, if I then run a scan through the Endpoint coprocessor (with a > filter just looking for the name "john"), I get the following results: > > , ,
, <123 main street> > , , , > , ,
, <345 scott street> > , , , > > It's filtered properly, but the values don't go through the Observer and > aren't capitalized. > > If there is any other info you need to help diagnose this, please let me > know. Thanks. > > - Bob Gaimari > > > > -- > View this message in context: > http://apache-hbase.679495.n3.nabble.com/Endpoint-and-Observer-work-together-tp4051383p4051395.html > Sent from the HBase User mailing list archive at Nabble.com. > --001a11c3faceb5104504e7c8ef52--