Return-Path: X-Original-To: apmail-accumulo-user-archive@www.apache.org Delivered-To: apmail-accumulo-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 6BAB21142F for ; Tue, 20 May 2014 20:34:39 +0000 (UTC) Received: (qmail 1597 invoked by uid 500); 20 May 2014 20:34:39 -0000 Delivered-To: apmail-accumulo-user-archive@accumulo.apache.org Received: (qmail 1542 invoked by uid 500); 20 May 2014 20:34:39 -0000 Mailing-List: contact user-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@accumulo.apache.org Delivered-To: mailing list user@accumulo.apache.org Received: (qmail 1534 invoked by uid 99); 20 May 2014 20:34:39 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 20 May 2014 20:34:39 +0000 X-ASF-Spam-Status: No, hits=-2.3 required=5.0 tests=RCVD_IN_DNSWL_MED,SPF_HELO_PASS,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of David.Slater@jhuapl.edu designates 128.244.251.36 as permitted sender) Received: from [128.244.251.36] (HELO pilot.jhuapl.edu) (128.244.251.36) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 20 May 2014 20:34:33 +0000 Received: from aplexcas1.dom1.jhuapl.edu (unknown [128.244.198.90]) by pilot.jhuapl.edu with smtp (TLS: TLSv1/SSLv3,128bits,RC4-MD5) id 6144_55f9_87fb5a90_7d68_42c8_bcde_ebec32046dee; Tue, 20 May 2014 16:34:07 -0400 Received: from aplesstripe.dom1.jhuapl.edu ([128.244.198.211]) by aplexcas1.dom1.jhuapl.edu ([128.244.198.90]) with mapi; Tue, 20 May 2014 16:34:08 -0400 From: "Slater, David M." To: "user@accumulo.apache.org" Date: Tue, 20 May 2014 16:34:06 -0400 Subject: RE: Improving Batchscanner Performance Thread-Topic: Improving Batchscanner Performance Thread-Index: Ac90YB1Q+SZAHuyXSky5CAnFIEiMogABmJsw Message-ID: References: <537B9188.9010302@gmail.com> <537B97EB.5050506@gmail.com> <537BA203.6060803@gmail.com> <537BAA1C.3020005@gmail.com> In-Reply-To: <537BAA1C.3020005@gmail.com> Accept-Language: en-US Content-Language: en-US X-MS-Has-Attach: X-MS-TNEF-Correlator: acceptlanguage: en-US Content-Type: text/plain; charset="us-ascii" Content-Transfer-Encoding: quoted-printable MIME-Version: 1.0 X-Virus-Checked: Checked by ClamAV on apache.org 10-100 entries per node (4 nodes total). Would changing the data table structure change the batchscanner performance= ? I'm using: row colFam colQual value bin|guid -- -- byte[] would it be faster/slower to use: row colFam colQual value bin guid -- byte[] The difference would be that the first would include everything as a Collec= tion of ranges, where the second would use a combination of ranges and sett= ing column families.=20 -----Original Message----- From: Josh Elser [mailto:josh.elser@gmail.com]=20 Sent: Tuesday, May 20, 2014 3:17 PM To: user@accumulo.apache.org Subject: Re: Improving Batchscanner Performance 10-100 entries/s seems slow, but that's mostly a gut feeling without contex= t. Is this over more than one node? 10s of nodes? A value of 1M would would explain the pause that you see in the beginning. = That parameter controls the size of the buffer that each tserver will fill = before sending data back to the BatchScanner. Too small and you pay for the= excessive RPCs, too large, and like you're seeing, it takes longer for you= to get the first batch. You should be able to reduce that value and see a = much quick first result come out of the batchscanner. Number of rfiles could impact read performance as you have to do a merged-r= ead over all of the rfiles for a tablet. On 5/20/14, 3:08 PM, Slater, David M. wrote: > I'm getting query results around 10-100 entries/s. However, it takes some= time after starting the data scan to actually have any positive query numb= er. The ingest rate into this table is about 10k entries/s. > > I don't think this would be a problem with table.scan.max.memory=3D1M, wo= uld it? > > Maybe it's a problem with the number of rfiles on disk? Or perhaps the in= gest is overwhelming the resources? > > -----Original Message----- > From: Josh Elser [mailto:josh.elser@gmail.com] > Sent: Tuesday, May 20, 2014 2:42 PM > To: user@accumulo.apache.org > Subject: Re: Improving Batchscanner Performance > > No, that is how it's done. The ranges that you provide to the BatchScanne= r are binned to tablets hosted by tabletserver. It will then query up to nu= mQueryThreads tservers at once to fetch results in parallel. > > The point I was making is that you can only bin ranges within the scope o= f a single BatchScanner, and if you were making repeated calls to your orig= inal function with differing arguments, you might be incurring some more pe= nalty. Like Bob, fetching random sets of rows and data is what I was trying= to lead you to. > > If the bandwidth of fetching the data is not a factor, I would probably a= gree that random reads are an issue. Do you have more details you can give = about how long it takes to fetch the data for N rows (e.g. number of key-va= lues/second and/or amount of data/second)? Are you getting an even distribu= tion across your tservers or hot-spotted on a few number (the monitor shoul= d help here)? It can sometimes be a bit of a balancing act with optimizing = locality while avoid suffering from hotspots. > > On 5/20/14, 2:24 PM, Slater, David M. wrote: >> Josh, >> >> The data is not significantly larger than the rows that I'm fetching. in= terms of bandwidth, the data returned is at least 2 orders of magnitude sm= aller than the ingest rate, so I don't think it's a network issue. >> >> I'm guessing, as Bob suggested, that it has to do with fetching a "rando= m" set of rows each time. I had assumed that the batchscanner would take th= e Collection of ranges (when setting batchScanner.setRanges()), sort them, = and then fetch data based on tablet splits. I'm guessing, based on the disc= ussion, that it is not done that way. >> >> Does the BatchScanner fetch rows based on the ordering of the Collection= ? >> >> Thanks, >> David >> >> -----Original Message----- >> From: Josh Elser [mailto:josh.elser@gmail.com] >> Sent: Tuesday, May 20, 2014 1:59 PM >> To: user@accumulo.apache.org >> Subject: Re: Improving Batchscanner Performance >> >> You actually stated it exactly here: >> >> > I complete the first scan in its entirety >> >> Loading the data into a Collection also implies that you're loading the = complete set of rows and blocking until you find all rows, or until you fet= ch all of the data. >> >> > Collection rows =3D getRowIDs(new Range("minRow", "maxRow"),= =20 >> new Text("index"), "mytable", 10, 10000); > Collection data=20 >> =3D getRowData(rows, "mytable", 10); >> >> Both the BatchScanner and Scanner are returning KeyValue pairs in "batch= es". The client talks to server(s), reads some data and returns it to you. = By virtue of you loading these results from the Iterator into a Collection,= you are consuming *all* results before proceeding to fetch the data for th= e rows. >> >> Now, if, like you said, looking up the rows is drastically faster than f= etching the data, there's a question as to why this is. Is it safe to assum= e that the data is much larger than the rows you're fetching? Have you trie= d to see what the throughput of fetching this data is? If it's bounded by n= etwork speed, you could try compressing the data in an iterator server-side= before returning it to the client. >> >> You could also consider the locality of the rows that you're fetching --= are you fetching a "random" set of rows each time and paying a penalty of = talking to each server to fetch the data when you could ammortize the cost = if you fetched the data for rows that are close together. A large amount of= data being returned is likely going to trump the additional cost in talkin= g to many servers. >> >> >> On 5/20/14, 1:51 PM, Slater, David M. wrote: >>> Hi Josh, >>> >>> I should have clarified - I am using a batchscanner for both lookups. I= had thought of putting it into two different threads, but the first scan i= s typically an order of magnitude faster than the second. >>> >>> The logic for upperbounding the results returned is outside of the meth= od I provided. Since there is a one-to-one relationship between rowIDs and = records on the second scan, I just limit the number of rows I send to this = method. >>> >>> As for blocking, I'm not sure exactly what you mean. I complete the fir= st scan in its entirety, which before entering this method with the collec= tion of Text rowIDs. The method for that is: >>> >>> public Collection getRowIDs(Collection ranges, Text term, = String tablename, int queryThreads, int limit) throws TableNotFoundExceptio= n { >>> Set guids =3D new HashSet(); >>> if (!ranges.isEmpty()) { >>> BatchScanner scanner =3D conn.createBatchScanner(tablena= me, new Authorizations(), queryThreads); >>> scanner.setRanges(ranges); >>> scanner.fetchColumnFamily(term); >>> for (Map.Entry entry : scanner) { >>> guids.add(entry.getKey().getColumnQualifier()); >>> if (guids.size() > limit) { >>> return null; >>> } >>> } >>> scanner.close(); >>> } >>> return guids; >>> } >>> >>> Essentially, my query does: >>> Collection rows =3D getRowIDs(new Range("minRow", "maxRow"), new= =20 >>> Text("index"), "mytable", 10, 10000); Collection data =3D=20 >>> getRowData(rows, "mytable", 10); >>> >>> >>> -----Original Message----- >>> From: Josh Elser [mailto:josh.elser@gmail.com] >>> Sent: Tuesday, May 20, 2014 1:32 PM >>> To: user@accumulo.apache.org >>> Subject: Re: Improving Batchscanner Performance >>> >>> Hi David, >>> >>> Absolutely. What you have here is a classic producer-consumer model. >>> Your BatchScanner is producing results, which you then consume by your = scanner, and ultimately return those results to the client. >>> >>> The problem with your below implementation is that you're not going to = be polling your batchscanner as aggressively as you could be. You are block= ing while you can fetch each of those new Ranges from the Scanner before fe= tching new ranges. Have you considered splitting up the BatchScanner and Sc= anner code into two different threads? >>> >>> You could easily use a ArrayBlockingQueue (or similar) to pass results = from the BatchScanner to the Scanner. I would imagine that this would give = you a fair improvement in performance. >>> >>> Also, it doesn't appear that there's a reason you can't use a BatchScan= ner for both lookups? >>> >>> One final warning, your current implementation could also hog heap very= badly if your batchscanner returns too many records. The producer/consumer= I proposed should help here a little bit, but you should still be assertin= g upper-bounds to avoid running out of heap space in your client. >>> >>> On 5/20/14, 1:10 PM, Slater, David M. wrote: >>>> Hey everyone, >>>> >>>> I'm trying to improve the query performance of batchscans on my data t= able. I first scan over index tables, which returns a set of rowIDs that co= rrespond to the records I am interested in. This set of records is fairly r= andomly (and uniformly) distributed across a large number of tablets, due t= o the randomness of the UID and the query itself. Then I want to scan over = my data table, which is setup as follows: >>>> row colFam colQual value >>>> rowUID -- -- byte[] of data >>>> >>>> These records are fairly small (100s of bytes), but numerous (I may re= turn 50000 or more). The method I use to obtain this follows. Essentially, = I turn the rows returned from the first query into a set of ranges to input= into the batchscanner, and then return those rows, retrieving the value fr= om them. >>>> >>>> // returns the data associated with the given collection of rows >>>> public Collection getRowData(Collection rows, Te= xt dataType, String tablename, int queryThreads) throws TableNotFoundExcept= ion { >>>> List values =3D new ArrayList(rows.size())= ; >>>> if (!rows.isEmpty()) { >>>> BatchScanner scanner =3D conn.createBatchScanner(table= name, new Authorizations(), queryThreads); >>>> List ranges =3D new ArrayList(); >>>> for (Text row : rows) { >>>> ranges.add(new Range(row)); >>>> } >>>> scanner.setRanges(ranges); >>>> for (Map.Entry entry : scanner) { >>>> values.add(entry.getValue().get()); >>>> } >>>> scanner.close(); >>>> } >>>> return values; >>>> } >>>> >>>> Is there a more efficient way to do this? I have index caches and bloo= m filters enabled (data caches are not), but I still seem to have a long qu= ery lag. Any thoughts on how I can improve this? >>>> >>>> Thanks, >>>> David >>>>