Re: Using HBase for Deduping
Michael, this means read for every write? On Friday, February 15, 2013, Michael Segel wrote: What constitutes a duplicate? An over simplification is to do a HTable.checkAndPut() where you do the put if the column doesn't exist. Then if the row is inserted (TRUE) return value, you push the event. That will do what you want. At least at first blush. On Feb 14, 2013, at 3:24 PM, Viral Bajaria viral.baja...@gmail.com wrote: Given the size of the data ( 1B rows) and the frequency of job run (once per hour), I don't think your most optimal solution is to lookup HBase for every single event. You will benefit more by loading the HBase table directly in your MR job. In 1B rows, what's the cardinality ? Is it 100M UUID's ? 99% unique UUID's ? Also once you have done the unique, are you going to use the data again in some other way i.e. online serving of traffic or some other analysis ? Or this is just to compute some unique #'s ? It will be more helpful if you describe your final use case of the computed data too. Given the amount of back and forth, we can take it off list too and summarize the conversation for the list. On Thu, Feb 14, 2013 at 1:07 PM, Rahul Ravindran rahu...@yahoo.com wrote: We can't rely on the the assumption event dupes will not dupe outside an hour boundary. So, your take is that, doing a lookup per event within the MR job is going to be bad? From: Viral Bajaria viral.baja...@gmail.com To: Rahul Ravindran rahu...@yahoo.com Cc: user@hbase.apache.org user@hbase.apache.org Sent: Thursday, February 14, 2013 12:48 PM Subject: Re: Using HBase for Deduping You could do with a 2-pronged approach here i.e. some MR and some HBase lookups. I don't think this is the best solution either given the # of events you will get. FWIW, the solution below again relies on the assumption that if a event is duped in the same hour it won't have a dupe outside of that hour boundary. If it can have then you are better of with running a MR job with the current hour + another 3 hours of data or an MR job with the current hour + the HBase table as input to the job too (i.e. no HBase lookups, just read the HFile directly) ? - Run a MR job which de-dupes events for the current hour i.e. only runs on 1 hour worth of data. - Mark records which you were not able to de-dupe in the current run - For the records that you were not able to de-dupe, check against HBase whether you saw that event in the past. If you did, you can drop the current event or update the event to the new value (based on your business logic) - Save all the de-duped events (via HBase bulk upload) Sorry if I just rambled along, but without knowing the whole problem it's very tough to come up with a probable solution. So correct my assumptions and we could drill down more. Thanks, Viral On Thu, Feb 14, 2013 at 12:29 PM, Rahul Ravindran rahu...@yahoo.com wrote: Most will be in the same hour. Some will be across 3-6 hours. Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 12:19 PM, Viral Bajaria viral.baja...@gmail.com wrote: Are all these dupe events expected to be within the same hour or they can happen over multiple hours ? Viral From: Rahul Ravindran Sent: 2/14/2013 11:41 AM To: user@hbase.apache.org Subject: Using HBase for Deduping Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each event has a UUID and we were hoping to leverage Michael Segel | (m) 312.755.9623 Segel and Associates
Re: hbase-master-server slept
Hi, We consider to update the version of Hbase. The VM spec is below. CPU:2 Core MEMORY:4GB We don't know the hardware spec of host server, because we owe the VM from a VPS provider. Additionally, I check vmstat for the time spanning the isuue. No batch and online job worked on this server for this time. But the cpu usage was very high suddenly. Is it a clue? 2013/01/17 16:05:51 procs ---memory-- ---swap-- -io --system-- -cpu-- 2013/01/17 16:05:51 r b swpd free buff cache si sobibo in cs us sy id wa st 2013/01/18 09:15:45 0 1 0 3113832 126920 57333200 0 3 1001 121 0 0 100 0 0 2013/01/18 09:15:50 0 1 0 3113832 126924 57333200 018 1002 125 0 0 100 0 0 2013/01/18 09:15:55 0 1 0 3113832 126924 57333200 0 2 1001 123 0 0 100 0 0 2013/01/18 09:16:00 0 1 0 3113832 126924 57333200 015 1001 122 0 0 100 0 0 2013/01/18 09:16:05 0 0 0 3113708 126940 57335200 035 1002 140 0 0 100 0 0 2013/01/18 09:16:10 0 0 0 3113708 126944 57335200 0 7 1001 122 0 0 100 0 0 2013/01/18 09:16:15 0 1 0 3113708 126944 57335200 0 6 1001 122 0 0 100 0 0 2013/01/18 09:16:20 0 1 0 3113708 126944 57335200 0 1 1001 122 0 0 100 0 0 2013/01/18 09:16:25 0 1 0 3113708 126948 57334800 0 7 1001 123 0 0 100 0 0 2013/01/18 09:16:30 0 1 0 3113708 126948 57335200 028 1001 130 0 0 100 0 0 2013/01/18 09:16:35 0 1 0 3113708 126948 57335200 012 1001 128 0 0 100 0 0 2013/01/18 09:16:40 0 1 0 3113708 126956 57335200 0 9 1001 127 0 0 100 0 0 2013/01/18 09:16:45 0 1 0 3113708 126956 57335200 0 5 1001 127 0 0 100 0 0 2013/01/18 09:16:50 0 1 0 3113708 126956 57335200 0 3 1001 128 0 0 100 0 0 2013/01/18 09:16:55 0 1 0 3113708 126964 57334800 0 6 1001 128 0 0 99 0 0 2013/01/18 09:17:00 0 1 0 3113708 126972 57334800 0 3 1001 127 0 0 100 0 0 2013/01/18 09:17:05 0 0 0 3113708 126976 57337200 025 1002 142 0 0 100 0 0 2013/01/18 09:17:10 0 0 0 3113708 126976 57337200 053 1002 128 0 0 100 0 0 2013/01/18 09:17:15 0 0 0 3113708 126980 57337200 011 1001 127 0 0 100 0 0 2013/01/18 09:17:20 0 0 0 3113708 126984 57336800 0 3 1001 126 0 0 100 0 0 2013/01/18 09:17:25 0 1 0 3113708 126992 57336400 0 6 1001 127 0 0 100 0 0 2013/01/18 09:17:30 0 1 0 3113708 126992 57337600 0 2 1001 125 0 0 100 0 0 2013/01/18 09:17:35 0 1 0 3113708 127000 57337200 010 1001 128 0 0 100 0 0 2013/01/18 09:17:40 0 1 0 3113708 127000 57337600 0 2 1001 126 0 0 100 0 0 2013/01/18 09:17:46 2 1 0 3113692 127004 57337600 011 806 29 2 2 95 2 0 2013/01/18 09:17:53 1 1 0 3113708 127008 57337600 010 1043 15 0 10 40 50 0 2013/01/18 09:18:00 1 1 0 3113708 127008 57337600 0 4 1364 15 8 0 67 25 0 2013/01/18 09:18:06 3 0 0 3113584 127008 57339600 026 1591 32 8 17 67 8 0 2013/01/18 09:18:12 1 0 0 3113584 127012 57339600 021 1004 10 0 0 89 11 0 2013/01/18 09:18:17 1 1 0 3113584 127020 57338800 034 1392 12 10 0 90 0 0 2013/01/18 09:18:53 2 2 0 3113584 127020 57339600 0 6 1075 15 0 25 75 0 0 2013/01/18 09:18:58 1 1 0 3113584 127020 57339600 014 6534 18 2 0 98 0 0 2013/01/18 09:19:04 2 0 0 3113576 127032 57339200 0 9 1604 20 0 9 27 64 0 2013/01/18 09:19:11 3 1 0 3113584 127044 57341600 041 1579 29 17 17 67 0 0 2013/01/18 09:19:17 1 1 0 3113584 127048 57341600 0 3 705 10 0 0 50 50 0 2013/01/18 09:19:23 1 1 0 3113584 127060 57341200 0 7 1684 15 0 6 88 6 0 2013/01/18 09:19:28 1 2 0 3113584 127060 57341600 033 1153 21 10 20 60 10 0 2013/01/18 09:19:33 1 1 0 3113584 127060 57341600 0 6 861 16 0 0 44 56 0 2013/01/18 09:19:39 1 1 0 3113584 127068 57341600 0 6 1036 16 0 0 78 22 0 2013/01/18 09:19:44 0 1 0 3113584 127072 57341600 0 3 1007 10 14 14 57 14 0 2013/01/18 09:19:49 0 1 0 3113584 127076 57341600 0 6 1369 18 0 0 91 9 0 2013/01/18 09:19:56 2 0 0 3113584 127084 57340800 0 8 596 11 20 20 60 0 0 2013/01/18 09:20:01 1 1 0 3113584 127084 57341600 0
Re: Using HBase for Deduping
Then maybe he can place an event in the same rowkey but with a column qualifier which the time stamp of the event saved as long. Upon preCompact in a region observer he can filter out for any row all column but the first? On Friday, February 15, 2013, Anoop Sam John wrote: When max versions set as 1 and duplicate key is added, the last added will win removing the old. This is what you want Rahul? I think from his explanation he needs the reverse way -Anoop- From: Asaf Mesika [asaf.mes...@gmail.com javascript:;] Sent: Friday, February 15, 2013 3:56 AM To: user@hbase.apache.org javascript:;; Rahul Ravindran Subject: Re: Using HBase for Deduping You can load the events into an Hbase table, which has the event id as the unique row key. You can define max versions of 1 to the column family thus letting Hbase get rid of the duplicates for you during major compaction. On Thursday, February 14, 2013, Rahul Ravindran wrote: Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each event has a UUID and we were hoping to leverage HBase to dedupe them. We run a MapReduce job which would perform a lookup for each UUID on HBase and then emit the event only if the UUID was absent and would also insert into the HBase table(This is simplistic, I am missing out details to make this more resilient to failures). My concern is that doing a Read+Write for every event in MR would be slow (We expect around 1 Billion events every hour). Does anyone use Hbase for a similar use case or is there a different approach to achieving the same end result. Any information, comments would be great. Thanks, ~Rahul.
RE: Using HBase for Deduping
Or may be go with large value for max version and put the duplicate entry. Now in the compact, need to have a wrapper for InternalScanner and next() method return only the 1st KV out, removing the others... Even while scan also same kind of logic will be needed.. This will be good enough IMO especially when there wont be so many duplicate events for same rowkey.. That is why I asked some questions before I think this solution can be checked. -Anoop- From: Asaf Mesika [asaf.mes...@gmail.com] Sent: Friday, February 15, 2013 3:06 PM To: user@hbase.apache.org Cc: Rahul Ravindran Subject: Re: Using HBase for Deduping Then maybe he can place an event in the same rowkey but with a column qualifier which the time stamp of the event saved as long. Upon preCompact in a region observer he can filter out for any row all column but the first? On Friday, February 15, 2013, Anoop Sam John wrote: When max versions set as 1 and duplicate key is added, the last added will win removing the old. This is what you want Rahul? I think from his explanation he needs the reverse way -Anoop- From: Asaf Mesika [asaf.mes...@gmail.com javascript:;] Sent: Friday, February 15, 2013 3:56 AM To: user@hbase.apache.org javascript:;; Rahul Ravindran Subject: Re: Using HBase for Deduping You can load the events into an Hbase table, which has the event id as the unique row key. You can define max versions of 1 to the column family thus letting Hbase get rid of the duplicates for you during major compaction. On Thursday, February 14, 2013, Rahul Ravindran wrote: Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each event has a UUID and we were hoping to leverage HBase to dedupe them. We run a MapReduce job which would perform a lookup for each UUID on HBase and then emit the event only if the UUID was absent and would also insert into the HBase table(This is simplistic, I am missing out details to make this more resilient to failures). My concern is that doing a Read+Write for every event in MR would be slow (We expect around 1 Billion events every hour). Does anyone use Hbase for a similar use case or is there a different approach to achieving the same end result. Any information, comments would be great. Thanks, ~Rahul.
Re: Using HBase for Deduping
On Feb 15, 2013, at 3:07 AM, Asaf Mesika asaf.mes...@gmail.com wrote: Michael, this means read for every write? Yes and no. At the macro level, a read for every write would mean that your client would read a record from HBase, and then based on some logic it would either write a record, or not. So that you have a lot of overhead in the initial get() and then put(). At this macro level, with a Check and Put you have less overhead because of a single message to HBase. Intermal to HBase, you would still have to check the value in the row, if it exists and then perform an insert or not. WIth respect to your billion events an hour... dividing by 3600 to get the number of events in a second. You would have less than 300,000 events a second. What exactly are you doing and how large are those events? Since you are processing these events in a batch job, timing doesn't appear to be that important and of course there is also async hbase which may improve some of the performance. YMMV but this is a good example of the checkAndPut() On Friday, February 15, 2013, Michael Segel wrote: What constitutes a duplicate? An over simplification is to do a HTable.checkAndPut() where you do the put if the column doesn't exist. Then if the row is inserted (TRUE) return value, you push the event. That will do what you want. At least at first blush. On Feb 14, 2013, at 3:24 PM, Viral Bajaria viral.baja...@gmail.com wrote: Given the size of the data ( 1B rows) and the frequency of job run (once per hour), I don't think your most optimal solution is to lookup HBase for every single event. You will benefit more by loading the HBase table directly in your MR job. In 1B rows, what's the cardinality ? Is it 100M UUID's ? 99% unique UUID's ? Also once you have done the unique, are you going to use the data again in some other way i.e. online serving of traffic or some other analysis ? Or this is just to compute some unique #'s ? It will be more helpful if you describe your final use case of the computed data too. Given the amount of back and forth, we can take it off list too and summarize the conversation for the list. On Thu, Feb 14, 2013 at 1:07 PM, Rahul Ravindran rahu...@yahoo.com wrote: We can't rely on the the assumption event dupes will not dupe outside an hour boundary. So, your take is that, doing a lookup per event within the MR job is going to be bad? From: Viral Bajaria viral.baja...@gmail.com To: Rahul Ravindran rahu...@yahoo.com Cc: user@hbase.apache.org user@hbase.apache.org Sent: Thursday, February 14, 2013 12:48 PM Subject: Re: Using HBase for Deduping You could do with a 2-pronged approach here i.e. some MR and some HBase lookups. I don't think this is the best solution either given the # of events you will get. FWIW, the solution below again relies on the assumption that if a event is duped in the same hour it won't have a dupe outside of that hour boundary. If it can have then you are better of with running a MR job with the current hour + another 3 hours of data or an MR job with the current hour + the HBase table as input to the job too (i.e. no HBase lookups, just read the HFile directly) ? - Run a MR job which de-dupes events for the current hour i.e. only runs on 1 hour worth of data. - Mark records which you were not able to de-dupe in the current run - For the records that you were not able to de-dupe, check against HBase whether you saw that event in the past. If you did, you can drop the current event or update the event to the new value (based on your business logic) - Save all the de-duped events (via HBase bulk upload) Sorry if I just rambled along, but without knowing the whole problem it's very tough to come up with a probable solution. So correct my assumptions and we could drill down more. Thanks, Viral On Thu, Feb 14, 2013 at 12:29 PM, Rahul Ravindran rahu...@yahoo.com wrote: Most will be in the same hour. Some will be across 3-6 hours. Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 12:19 PM, Viral Bajaria viral.baja...@gmail.com wrote: Are all these dupe events expected to be within the same hour or they can happen over multiple hours ? Viral From: Rahul Ravindran Sent: 2/14/2013 11:41 AM To: user@hbase.apache.org Subject: Using HBase for Deduping Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each event has a UUID and we were hoping to leverage Michael Segel | (m) 312.755.9623 Segel and Associates
Re: Using HBase for Deduping
But then he can't trigger an event if its a net new row. Methinks that he needs to better define the problem he is trying to solve. Also the number of events. A billion an hour or 300K events a second? (Ok its 277.78K events a second.) On Feb 14, 2013, at 10:19 PM, Anoop Sam John anoo...@huawei.com wrote: When max versions set as 1 and duplicate key is added, the last added will win removing the old. This is what you want Rahul? I think from his explanation he needs the reverse way -Anoop- From: Asaf Mesika [asaf.mes...@gmail.com] Sent: Friday, February 15, 2013 3:56 AM To: user@hbase.apache.org; Rahul Ravindran Subject: Re: Using HBase for Deduping You can load the events into an Hbase table, which has the event id as the unique row key. You can define max versions of 1 to the column family thus letting Hbase get rid of the duplicates for you during major compaction. On Thursday, February 14, 2013, Rahul Ravindran wrote: Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each event has a UUID and we were hoping to leverage HBase to dedupe them. We run a MapReduce job which would perform a lookup for each UUID on HBase and then emit the event only if the UUID was absent and would also insert into the HBase table(This is simplistic, I am missing out details to make this more resilient to failures). My concern is that doing a Read+Write for every event in MR would be slow (We expect around 1 Billion events every hour). Does anyone use Hbase for a similar use case or is there a different approach to achieving the same end result. Any information, comments would be great. Thanks, ~Rahul.
Re: question about pre-splitting regions
Good to hear! Given your experience, I'd appreciate your feedback on the section 6.3.6. Relationship Between RowKeys and Region Splits in... http://hbase.apache.org/book.html#schema.creation Š because it's on that same topic. Any other points to add to this? Thanks! On 2/14/13 11:08 PM, Viral Bajaria viral.baja...@gmail.com wrote: I was able to figure it out. I had to use the createTable api which took splitKeys instead of the startKey, endKey and numPartitions. If anyone comes across this issue and needs more feedback feel free to ping me. Thanks, Viral On Thu, Feb 14, 2013 at 7:30 PM, Viral Bajaria viral.baja...@gmail.comwrote: Hi, I am creating a new table and want to pre-split the regions and am seeing some weird behavior. My table is designed as a composite of multiple fixed length byte arrays separated by a control character (for simplicity sake we can say the separator is _underscore_). The prefix of this rowkey is deterministic (i.e. length of 8 bytes) and I know it beforehand how many different prefix I will see in the near future. The values after the prefix is not deterministic. I wanted to create a pre-split tables based on the number of number of prefix combinations that I know. I ended up doing something like this: hbaseAdmin.createTable(tableName, Bytes.toBytes(1L), Bytes.toBytes(maxCombinationPrefixValue), maxCombinationPrefixValue) The create table worked fine and as expected it created the number of partitions. But when I write data to the table, I still see all the writes hitting a single region instead of hitting different regions based on the prefix. Is my thinking of splitting by prefix values flawed ? Do I have to split by some real rowkeys (though it's impossible for me to know what rowkeys will show up except the row prefix which is much more deterministic). For some reason I think I have a flawed understanding of the createTable API and that is causing the issue for me ? Should I use the byte[][] prefixes method and not the one that I am using right now ? Any suggestions/pointers ? Thanks, Viral
Re: Using HBase for Deduping
I had tried checkAndPut yesterday with a null passed as the value and it had thrown an exception when the row did not exist. Perhaps, I was doing something wrong. Will try that again, since, yes, I would prefer a checkAndPut(). From: Michael Segel michael_se...@hotmail.com To: user@hbase.apache.org Cc: Rahul Ravindran rahu...@yahoo.com Sent: Friday, February 15, 2013 4:36 AM Subject: Re: Using HBase for Deduping On Feb 15, 2013, at 3:07 AM, Asaf Mesika asaf.mes...@gmail.com wrote: Michael, this means read for every write? Yes and no. At the macro level, a read for every write would mean that your client would read a record from HBase, and then based on some logic it would either write a record, or not. So that you have a lot of overhead in the initial get() and then put(). At this macro level, with a Check and Put you have less overhead because of a single message to HBase. Intermal to HBase, you would still have to check the value in the row, if it exists and then perform an insert or not. WIth respect to your billion events an hour... dividing by 3600 to get the number of events in a second. You would have less than 300,000 events a second. What exactly are you doing and how large are those events? Since you are processing these events in a batch job, timing doesn't appear to be that important and of course there is also async hbase which may improve some of the performance. YMMV but this is a good example of the checkAndPut() On Friday, February 15, 2013, Michael Segel wrote: What constitutes a duplicate? An over simplification is to do a HTable.checkAndPut() where you do the put if the column doesn't exist. Then if the row is inserted (TRUE) return value, you push the event. That will do what you want. At least at first blush. On Feb 14, 2013, at 3:24 PM, Viral Bajaria viral.baja...@gmail.com wrote: Given the size of the data ( 1B rows) and the frequency of job run (once per hour), I don't think your most optimal solution is to lookup HBase for every single event. You will benefit more by loading the HBase table directly in your MR job. In 1B rows, what's the cardinality ? Is it 100M UUID's ? 99% unique UUID's ? Also once you have done the unique, are you going to use the data again in some other way i.e. online serving of traffic or some other analysis ? Or this is just to compute some unique #'s ? It will be more helpful if you describe your final use case of the computed data too. Given the amount of back and forth, we can take it off list too and summarize the conversation for the list. On Thu, Feb 14, 2013 at 1:07 PM, Rahul Ravindran rahu...@yahoo.com wrote: We can't rely on the the assumption event dupes will not dupe outside an hour boundary. So, your take is that, doing a lookup per event within the MR job is going to be bad? From: Viral Bajaria viral.baja...@gmail.com To: Rahul Ravindran rahu...@yahoo.com Cc: user@hbase.apache.org user@hbase.apache.org Sent: Thursday, February 14, 2013 12:48 PM Subject: Re: Using HBase for Deduping You could do with a 2-pronged approach here i.e. some MR and some HBase lookups. I don't think this is the best solution either given the # of events you will get. FWIW, the solution below again relies on the assumption that if a event is duped in the same hour it won't have a dupe outside of that hour boundary. If it can have then you are better of with running a MR job with the current hour + another 3 hours of data or an MR job with the current hour + the HBase table as input to the job too (i.e. no HBase lookups, just read the HFile directly) ? - Run a MR job which de-dupes events for the current hour i.e. only runs on 1 hour worth of data. - Mark records which you were not able to de-dupe in the current run - For the records that you were not able to de-dupe, check against HBase whether you saw that event in the past. If you did, you can drop the current event or update the event to the new value (based on your business logic) - Save all the de-duped events (via HBase bulk upload) Sorry if I just rambled along, but without knowing the whole problem it's very tough to come up with a probable solution. So correct my assumptions and we could drill down more. Thanks, Viral On Thu, Feb 14, 2013 at 12:29 PM, Rahul Ravindran rahu...@yahoo.com wrote: Most will be in the same hour. Some will be across 3-6 hours. Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 12:19 PM, Viral Bajaria viral.baja...@gmail.com wrote: Are all these dupe events expected to be within the same hour or they can happen over multiple hours ? Viral From: Rahul Ravindran Sent: 2/14/2013 11:41 AM To: user@hbase.apache.org Subject: Using HBase for Deduping Hi, We have events which are delivered into our HDFS cluster which may be duplicated. Each
Re: Using HBase for Deduping
Interesting. Surround with a Try Catch? But it sounds like you're on the right path. Happy Coding! On Feb 15, 2013, at 11:12 AM, Rahul Ravindran rahu...@yahoo.com wrote: I had tried checkAndPut yesterday with a null passed as the value and it had thrown an exception when the row did not exist. Perhaps, I was doing something wrong. Will try that again, since, yes, I would prefer a checkAndPut(). From: Michael Segel michael_se...@hotmail.com To: user@hbase.apache.org Cc: Rahul Ravindran rahu...@yahoo.com Sent: Friday, February 15, 2013 4:36 AM Subject: Re: Using HBase for Deduping On Feb 15, 2013, at 3:07 AM, Asaf Mesika asaf.mes...@gmail.com wrote: Michael, this means read for every write? Yes and no. At the macro level, a read for every write would mean that your client would read a record from HBase, and then based on some logic it would either write a record, or not. So that you have a lot of overhead in the initial get() and then put(). At this macro level, with a Check and Put you have less overhead because of a single message to HBase. Intermal to HBase, you would still have to check the value in the row, if it exists and then perform an insert or not. WIth respect to your billion events an hour... dividing by 3600 to get the number of events in a second. You would have less than 300,000 events a second. What exactly are you doing and how large are those events? Since you are processing these events in a batch job, timing doesn't appear to be that important and of course there is also async hbase which may improve some of the performance. YMMV but this is a good example of the checkAndPut() On Friday, February 15, 2013, Michael Segel wrote: What constitutes a duplicate? An over simplification is to do a HTable.checkAndPut() where you do the put if the column doesn't exist. Then if the row is inserted (TRUE) return value, you push the event. That will do what you want. At least at first blush. On Feb 14, 2013, at 3:24 PM, Viral Bajaria viral.baja...@gmail.com wrote: Given the size of the data ( 1B rows) and the frequency of job run (once per hour), I don't think your most optimal solution is to lookup HBase for every single event. You will benefit more by loading the HBase table directly in your MR job. In 1B rows, what's the cardinality ? Is it 100M UUID's ? 99% unique UUID's ? Also once you have done the unique, are you going to use the data again in some other way i.e. online serving of traffic or some other analysis ? Or this is just to compute some unique #'s ? It will be more helpful if you describe your final use case of the computed data too. Given the amount of back and forth, we can take it off list too and summarize the conversation for the list. On Thu, Feb 14, 2013 at 1:07 PM, Rahul Ravindran rahu...@yahoo.com wrote: We can't rely on the the assumption event dupes will not dupe outside an hour boundary. So, your take is that, doing a lookup per event within the MR job is going to be bad? From: Viral Bajaria viral.baja...@gmail.com To: Rahul Ravindran rahu...@yahoo.com Cc: user@hbase.apache.org user@hbase.apache.org Sent: Thursday, February 14, 2013 12:48 PM Subject: Re: Using HBase for Deduping You could do with a 2-pronged approach here i.e. some MR and some HBase lookups. I don't think this is the best solution either given the # of events you will get. FWIW, the solution below again relies on the assumption that if a event is duped in the same hour it won't have a dupe outside of that hour boundary. If it can have then you are better of with running a MR job with the current hour + another 3 hours of data or an MR job with the current hour + the HBase table as input to the job too (i.e. no HBase lookups, just read the HFile directly) ? - Run a MR job which de-dupes events for the current hour i.e. only runs on 1 hour worth of data. - Mark records which you were not able to de-dupe in the current run - For the records that you were not able to de-dupe, check against HBase whether you saw that event in the past. If you did, you can drop the current event or update the event to the new value (based on your business logic) - Save all the de-duped events (via HBase bulk upload) Sorry if I just rambled along, but without knowing the whole problem it's very tough to come up with a probable solution. So correct my assumptions and we could drill down more. Thanks, Viral On Thu, Feb 14, 2013 at 12:29 PM, Rahul Ravindran rahu...@yahoo.com wrote: Most will be in the same hour. Some will be across 3-6 hours. Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 12:19 PM, Viral Bajaria viral.baja...@gmail.com wrote: Are all these dupe events expected to be within the same hour or they can happen over multiple hours
queries and MR jobs
Hi Is there a way to partition HDFS [replication factor, say 3]] or route requests to specific RS nodes so that One set of nodes serve operations like put and get etc. Other set of nodes do MR on the same replicated data set And those two sets don't share the same nodes? I mean, If we are replicating and not worried about consistency equally across all replicas, can we allocate different jobs to different replicas based on that replica's consistency tuning. I understand that HDFS interleaves replicated data across nodes so we don't have cookie-cut isolated replicas. And thus this question becomes more interesting? :) An underlying question is how a node of its 2 other replicas, gets chosen for a specific request[ put/get] or a MR job. Thanks, Abhishek
Re: storing lists in columns
Hi Stas, Few options are coming into my mind. Quickly: 1) Why not storing the products in specif columns instead of in the same one? Like: table, rowid1, cf:list, c:aa, value:true table, rowid1, cf:list, c:bb, value:true table, rowid1, cf:list, c:cc, value:true table, rowid2, cf:list, c:aabb, value:true table, rowid2, cf:list, c:cc, value:true That way when you do a search you query directly the right column for the right row. And using exist call with also reduce the size of the data transfered. 2) You can store the data in the oposite way. Like: table, aa, cf:products, c:rowid1, value:true table, aabb, cf:products, c:rowid2, value:true table, bb, cf:products, c:rowid1, value:true table, cc, cf:products, c:rowid1, value:true table, cc, cf:products, c:rowid2, value:true Here, you query by your product ID, and you search the column based on your previous rowid. I will say the 2 solutions are equivalent, but it will really depend on your data pattern and you query pattern. JM 2013/2/15, Stas Maksimov maksi...@gmail.com: Hi all, I have a requirement to store lists in HBase columns like this: table, rowid1, f:list, aa, bb, cc table, rowid2, f:list, aabb, cc There is a further requirement to be able to find rows where f:list contains a particular item, e.g. when I need to find rows having item aa only rowid1 should match, and for item cc both rowid1 and rowid2 should match. For now I decided to use SingleColumnValueFilter with substring matching. As using comma-separated list proved difficult to search through, I'm using pipe symbols to separate items like this: |aa|bb|cc|, so that I could pass the search item surrounded by pipes into the filter: SingleColumnValueFilter ('f', 'list', =, 'substring:|aa|') This proved to work effectively enough, however I would prefer to use something more standard for my list storage (e.g. serialised JSON), or perhaps something even more optimised for a search - performance really does matter here. Any opinions on this solution and possible enhancements are much appreciated. Many thanks, Stas
Re: debugging responseTooSlow
The slow response took about 1.5 minutes. During this period, did you observe high latency ? If you have Ganglia installed on master / NN node, do you observe abnormal spike ? BTW did you presplit your table ? Thanks On Fri, Feb 15, 2013 at 7:14 PM, Viral Bajaria viral.baja...@gmail.comwrote: Hi, (using hbase 0.94.4 and hadoop 1.0.4) I have been seeing a lot of the following WARN in my logs: 2013-02-16 02:37:11,409 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: Stats: total=25.18 MB, free=2.97 GB, max=3 GB, blocks=1, accesses=52, hits=51, hitRatio=98.07%, , cachingAccesses=52, cachingHits=51, cachingHitsRatio=98.07%, , evictions=0, evicted=0, evictedPerRun=NaN 2013-02-16 02:37:33,368 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97509,call:multi(org.apache.hadoop.hbase.client.MultiAction@1c3308bd ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41009 ,starttimems:1360982155855,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:38:37,377 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97191,call:multi(org.apache.hadoop.hbase.client.MultiAction@3eafc7ae ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41014 ,starttimems:1360982220183,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:39:29,842 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:85300,call:multi(org.apache.hadoop.hbase.client.MultiAction@3d615428 ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41017 ,starttimems:1360982284538,queuetimems:1,class:HRegionServer,responsesize:0,method:multi} It's strange because this is a new hbase setup with almost no traffic on it. I am running a perf test and would not expect this to happen. The regionservers have 12GB heap space and are only using 1GB when that error happens. I just pushed close to 33K rows via a batch and I see the responseTooSlow. I enabled GC logging, but I don't see any GC lockups, and each GC attempt is only taking a few 100 ms. What else could be happening here, any pointers on debugging ? My setup is 1 Master running with 1 NN (on the same server) with 3 regionservers running alongside the datanodes. Thanks, Viral
Re: debugging responseTooSlow
If you take a look at sar from 2013-02-16 on 10.149.10.10http://10.149.10.10:41017/ do you see any major I/O wait, swapping, or anything out of the norm? Is this occurring on all three region servers? When the perf test is running can you verify you are writing to all three nodes? On Fri, Feb 15, 2013 at 11:03 PM, Ted Yu yuzhih...@gmail.com wrote: The slow response took about 1.5 minutes. During this period, did you observe high latency ? If you have Ganglia installed on master / NN node, do you observe abnormal spike ? BTW did you presplit your table ? Thanks On Fri, Feb 15, 2013 at 7:14 PM, Viral Bajaria viral.baja...@gmail.com wrote: Hi, (using hbase 0.94.4 and hadoop 1.0.4) I have been seeing a lot of the following WARN in my logs: 2013-02-16 02:37:11,409 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: Stats: total=25.18 MB, free=2.97 GB, max=3 GB, blocks=1, accesses=52, hits=51, hitRatio=98.07%, , cachingAccesses=52, cachingHits=51, cachingHitsRatio=98.07%, , evictions=0, evicted=0, evictedPerRun=NaN 2013-02-16 02:37:33,368 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97509,call:multi(org.apache.hadoop.hbase.client.MultiAction@1c3308bd ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41009 ,starttimems:1360982155855,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:38:37,377 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97191,call:multi(org.apache.hadoop.hbase.client.MultiAction@3eafc7ae ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41014 ,starttimems:1360982220183,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:39:29,842 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:85300,call:multi(org.apache.hadoop.hbase.client.MultiAction@3d615428 ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41017 ,starttimems:1360982284538,queuetimems:1,class:HRegionServer,responsesize:0,method:multi} It's strange because this is a new hbase setup with almost no traffic on it. I am running a perf test and would not expect this to happen. The regionservers have 12GB heap space and are only using 1GB when that error happens. I just pushed close to 33K rows via a batch and I see the responseTooSlow. I enabled GC logging, but I don't see any GC lockups, and each GC attempt is only taking a few 100 ms. What else could be happening here, any pointers on debugging ? My setup is 1 Master running with 1 NN (on the same server) with 3 regionservers running alongside the datanodes. Thanks, Viral -- Kevin O'Dell Customer Operations Engineer, Cloudera
[ANNOUNCE] HBase 0.94.5 is available for download
The HBase Team is pleased to announce the release of HBase 0.94.5. Download it from your favorite Apache mirror [1]. HBase 0.94.5 is a bug fix release and has 76 issues resolved against it. 0.94.5 is the current stable release of HBase. All previous 0.92.x and 0.94.x releases can upgraded to 0.94.5 via a rolling upgrade without downtime, intermediary versions can be skipped. For a complete list of changes, see release notes [2]. Yours, The HBase Team P.S. Thanks go out to the 23 individuals who contributed to this release! 1. http://www.apache.org/dyn/closer.cgi/hbase/ 2. https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12310753version=12323874
Re: debugging responseTooSlow
Viral: Did you use YCSB or LoadTestTool ? Was the load spread relatively evenly across your servers ? Thanks On Fri, Feb 15, 2013 at 9:19 PM, Viral Bajaria viral.baja...@gmail.comwrote: Yeah I noticed very high latency around the time of slow response, basically my client timed out for those requests. I have pre-split the table into 128 regions. Unfortunately I didn't have ganglia installed, I will install ganglia on those boxes and run the perf again and post the results. Regarding the I/O wait, the timeout only happened on one box or that's what I saw in the logs. When I run the test again with ganglia on, I will verify if it only happens on one node. Thanks, Viral On Fri, Feb 15, 2013 at 8:09 PM, Kevin O'dell kevin.od...@cloudera.com wrote: If you take a look at sar from 2013-02-16 on 10.149.10.10http://10.149.10.10:41017/ do you see any major I/O wait, swapping, or anything out of the norm? Is this occurring on all three region servers? When the perf test is running can you verify you are writing to all three nodes? On Fri, Feb 15, 2013 at 11:03 PM, Ted Yu yuzhih...@gmail.com wrote: The slow response took about 1.5 minutes. During this period, did you observe high latency ? If you have Ganglia installed on master / NN node, do you observe abnormal spike ? BTW did you presplit your table ? Thanks On Fri, Feb 15, 2013 at 7:14 PM, Viral Bajaria viral.baja...@gmail.com wrote: Hi, (using hbase 0.94.4 and hadoop 1.0.4) I have been seeing a lot of the following WARN in my logs: 2013-02-16 02:37:11,409 DEBUG org.apache.hadoop.hbase.io.hfile.LruBlockCache: Stats: total=25.18 MB, free=2.97 GB, max=3 GB, blocks=1, accesses=52, hits=51, hitRatio=98.07%, , cachingAccesses=52, cachingHits=51, cachingHitsRatio=98.07%, , evictions=0, evicted=0, evictedPerRun=NaN 2013-02-16 02:37:33,368 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97509,call:multi(org.apache.hadoop.hbase.client.MultiAction@1c3308bd ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41009 ,starttimems:1360982155855,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:38:37,377 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:97191,call:multi(org.apache.hadoop.hbase.client.MultiAction@3eafc7ae ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41014 ,starttimems:1360982220183,queuetimems:0,class:HRegionServer,responsesize:0,method:multi} 2013-02-16 02:39:29,842 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {processingtimems:85300,call:multi(org.apache.hadoop.hbase.client.MultiAction@3d615428 ), rpc version=1, client version=29, methodsFingerPrint=-1368823753,client: 10.149.10.10:41017 ,starttimems:1360982284538,queuetimems:1,class:HRegionServer,responsesize:0,method:multi} It's strange because this is a new hbase setup with almost no traffic on it. I am running a perf test and would not expect this to happen. The regionservers have 12GB heap space and are only using 1GB when that error happens. I just pushed close to 33K rows via a batch and I see the responseTooSlow. I enabled GC logging, but I don't see any GC lockups, and each GC attempt is only taking a few 100 ms. What else could be happening here, any pointers on debugging ? My setup is 1 Master running with 1 NN (on the same server) with 3 regionservers running alongside the datanodes. Thanks, Viral -- Kevin O'Dell Customer Operations Engineer, Cloudera