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 2103B10723 for ; Tue, 30 Apr 2013 16:27:34 +0000 (UTC) Received: (qmail 36423 invoked by uid 500); 30 Apr 2013 16:27:32 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 36296 invoked by uid 500); 30 Apr 2013 16:27:31 -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 Delivered-To: moderator for user@hbase.apache.org Received: (qmail 91924 invoked by uid 99); 30 Apr 2013 14:58:39 -0000 X-ASF-Spam-Status: No, hits=0.3 required=5.0 tests=FREEMAIL_REPLY,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of richard.hallier@gmail.com designates 209.85.212.174 as permitted sender) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=x-received:from:sender:to:references:in-reply-to:subject:date :message-id:mime-version:content-type:content-transfer-encoding :x-mailer:thread-index:content-language; bh=3VMMp3zKKWzEifpUMA8TlbeCq0/5kpvbfDtLNVREZOA=; b=Qu7F4qPjTseTptRXqMhWlnJ66h60bpdPLdBJl/X2PVm5lpAkSU9r6IGdNl9mN0PiD9 fF6eXLEB6eBQHWYqm6X33uXOerMV1+l+KdAnFTPZUnbOEHIHhKkXq4sD7G/WepbIeByD xf3vK5YgmQAAKbhWjmyL3PRsFmfihsrcOSrTrk4WZRegATlUrNYEzwxg0JuJl8+Le0QV kfMrn0wRBrkx/GKzxdz/KoeFqUVU6du2yejaUBu4r0YopzkvIou27ay48N0+XLESP4jW Pw/d6pvFn7fPkLzTKzS8LwlCNt6czjkNVyMBDytN1RyGLQZSKOn6D/KnDa9a//rDKlpG ZOZQ== X-Received: by 10.194.58.243 with SMTP id u19mr64102679wjq.47.1367333892673; Tue, 30 Apr 2013 07:58:12 -0700 (PDT) From: Sender: Richard HALLIER To: References: <1066821727.24555.1367247785075.JavaMail.www@wwinf8306> <1170978955.34203.1367255106041.JavaMail.www@wwinf8229> In-Reply-To: Subject: RE: Read access pattern Date: Tue, 30 Apr 2013 16:58:11 +0200 Message-ID: <007c01ce45b3$2332d700$69988500$@laposte.net> MIME-Version: 1.0 Content-Type: text/plain; charset="utf-8" Content-Transfer-Encoding: quoted-printable X-Mailer: Microsoft Outlook 14.0 Thread-Index: AQIDveLC/CNci/N6rDzR/B0X7JU/RAKiiJnWAg3jYD0CHSX0CAJ+eJ9umDluPYA= Content-Language: fr X-Virus-Checked: Checked by ClamAV on apache.org 1. Change the schema If I understand correctly, in this scenario, I loose the ordering = (changeDate desc). Moreover in my case, I could have 100k rows per = objectId, meaning I would have to iterate a long list, but I understand = the logic. If I only look for 24 hours before the original column hour, it maybe = should be simpler to just play with the current rowkey pattern : = hash(objectid)+LONG.MAX_VALUE-changeDate.getTime()-1000*3600*24 and = iterate to find the last item before reaching the current target. Or = maybe using also a filter on the regionserver to achieve this ? 2.=20 Whaow I feel weak to go this way :) -----Message d'origine----- De : Asaf Mesika [mailto:asaf.mesika@gmail.com]=20 Envoy=C3=A9 : mardi 30 avril 2013 07:50 =C3=80 : user@hbase.apache.org; ricla@laposte.net Objet : Re: Read access pattern Couple of raw implementation thoughts: 1. Change the schema Take the timestamps inside the row. Rowkey is the hash(objectid), and = column qualifier is the LONG.MAX_VALUE - changeDate - getTime(). You can = even save it using Bytes.toBytes(ts) to save space - will always be 8 = bytes, instead of the longer bytes string. This will enable you to "view" all the timestamps related to a single = objectid in one place. The problem with placing TS in the rowkey is that = it's all over the place - spread across regions, so it's harder to get a = valid who is before who response (indexing), without paying a penalty on = insertion for keeping it up to date. I have two ideas - one is expensive read and the other is expensive = write. Expensive read: When you write, you write two columns for that row: one named = i_[Rounded-to-the-hour-timestamp] with value of 1 (dummy value), = indicating you have timestamps with this hour, and the other is your = original column named ts_[timestamp]. You can implement a Filter, which upon arriving at the required row, = will first start by reading all "hour" timestamps, so it can find out = where to jump in the ts_[timestamp] column. Upon arriving to the = required hour timestamp matching the one you are looking for, you can = know which hour was before it, thus you can jump to it (using the hint = method in the Filter interface). The read is expensive since you need to = read all i_[Rounded-to-the-hour-timestamp] columns in the worst case. = Maybe you relax it by saying I only look for 24 hours before the = original column hour, thus reducing it only to 24 read worst case. The write is cheap, the read is not. Expensive write: You can keep a column named i, which maintains an encoded version of an = index for the hours, thus when you read, you achieve the correct before = hour on log(n) searching through it and then jump to the ts_[timestamp] = column. The write will be expensive, since you need to read-modify-write this = column on each timestamp you write. The read is sort of cheap. 2. I though I had another option of using RegionObserver and = EndpointCoprocessor but the biggest problem is the the predecessor = timestamp may be in another region server. The first idea is more = implementable :) On Mon, Apr 29, 2013 at 8:05 PM, wrote: > > Thanx for the quick answer. > > > For the next key, I think you can simply use your current key as=20 > > your scanner first key. You will then find the one which is just = after. > > Then you will have to verify the MD5 hash to make sure it's still=20 > > for the same object. > Right, this is basically easy. > > > First, if you know that you are storing data about every 10 seconds, = > > set the startRow with something like > > getMD5AsHex(Bytes.toBytes(myObjectId)) + String.format("%19d\n",=20 > > (Long.MAX_VALUE - (changeDate.getTime() - 60000))) then ready the=20 > > few lines you will have until you find your current line, and keep=20 > > the last one. > > Actually it is impossible to know the timerange for which there will=20 > be a next entry > > > > > Else, if you don't know, you will have to start with=20 > > scan.setStartRow(getMD5AsHex(Bytes.toBytes(myObjectId))); but you=20 > > might have to skip MANY lines before finding the right one. Do I=20 > > don't really recommend that. > > ouch, obviously not very efficient. I assume even with a filter ? > > Message du 29/04/13 18:18 > > De : "Jean-Marc Spaggiari" > > A : user@hbase.apache.org > > Copie =C3=A0 : > > Objet : Re: Read access pattern > > > > Hum. > > > > For the next key, I think you can simply use your current key as=20 > > your scanner first key. You will then find the one which is just = after. > > Then you will have to verify the MD5 hash to make sure it's still=20 > > for the same object. > > > > scan.setStartRow(getMD5AsHex(Bytes.toBytes(myObjectId)) +=20 > > String.format("%19d\n", (Long.MAX_VALUE - changeDate.getTime()))); > > > > If you want to find the one just before, quickly, I see 2 options. > > > > First, if you know that you are storing data about every 10 seconds, = > > set the startRow with something like > > getMD5AsHex(Bytes.toBytes(myObjectId)) + String.format("%19d\n",=20 > > (Long.MAX_VALUE - (changeDate.getTime() - 60000))) then ready the=20 > > few lines you will have until you find your current line, and keep=20 > > the last one. > > > > Else, if you don't know, you will have to start with=20 > > scan.setStartRow(getMD5AsHex(Bytes.toBytes(myObjectId))); but you=20 > > might have to skip MANY lines before finding the right one. Do I=20 > > don't really recommend that. > > > > JM > > > > 2013/4/29 Shahab Yunus : > > > I think you cannot use the scanner simply to to a range scan here=20 > > > as > your > > > keys are not monotonically increasing. You need to apply logic to=20 > > > decode/reverse your mechanism that you have used to hash your keys = > > > at > the > > > time of writing. You might want to check out the SemaText library=20 > > > which does distributed scans and seem to handle the scenarios that = > > > you want > to > > > implement. > > > > http://blog.sematext.com/2012/04/09/hbasewd-avoid-regionserver-hotspot > ting-despite-writing-records-with-sequential-keys/ > > > > > > > > > On Mon, Apr 29, 2013 at 11:03 AM, wrote: > > > > > >> Hi, > > >> > > >> I have a rowkey defined by : > > >> getMD5AsHex(Bytes.toBytes(myObjectId)) + String.format("%19d\n",=20 > > >> (Long.MAX_VALUE - changeDate.getTime())); > > >> > > >> How could I get the previous and next row for a given rowkey ? > > >> For instance, I have the following ordered keys : > > >> > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370673172227807 > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674468022807 > > >> >00003db1b6c1e7e7d2ece41ff2184f76*9223370674468862807 > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674984237807 > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674987271807 > > >> > > >> If I choose the rowkey : > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674468862807, what would=20 > > >> be > the > > >> correct scan to get the previous and next key ? > > >> Result would be : > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674468022807 > > >> 00003db1b6c1e7e7d2ece41ff2184f76*9223370674984237807 > > >> > > >> Thank you ! > > >> R. > > >> > > >> Une messagerie gratuite, garantie =C3=A0 vie et des services en = plus,=20 > > >> =C3=A7a > vous > > >> tente ? > > >> Je cr=C3=A9e ma bo=C3=AEte mail www.laposte.net > > >> > > > > Une messagerie gratuite, garantie =C3=A0 vie et des services en plus, = =C3=A7a=20 > vous tente ? > Je cr=C3=A9e ma bo=C3=AEte mail www.laposte.net >