Re: Using HBase for Deduping
I could surround with a Try..Catch, but that would each time I insert a UUID for the first time (99% of the time), I would do a checkAndPut(), catch the resultant exception and perform a Put; so, 2 operations each reduce invocation, which is what I was looking to avoid From: Michael Segel michael_se...@hotmail.com To: user@hbase.apache.org; Rahul Ravindran rahu...@yahoo.com Sent: Friday, February 15, 2013 9:24 AM Subject: 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
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: 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: 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
RE: Using HBase for Deduping
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 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
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 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
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 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
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 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
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 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
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 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. Michael Segel | (m) 312.755.9623 Segel and Associates
Re: Using HBase for Deduping
Checkandput() does not work when the row does not exist, or am I missing something? Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 5:33 PM, Michael Segel michael_se...@hotmail.com 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 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. Michael Segel | (m) 312.755.9623 Segel and Associates
Re: Using HBase for Deduping
Well, Maybe its a lack of sleep, but this is what I found... checkAndPut public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) throws IOException Atomically checks if a row/family/qualifier value matches the expected value. If it does, it adds the put. If the passed value is null, the check is for the lack of column (ie: non-existance) Specified by: checkAndPut in interface HTableInterface Parameters: row - to check family - column family to check qualifier - column qualifier to check value - the expected value put - data to put if check succeeds Returns: true if the new put was executed, false otherwise Throws: IOException - e Maybe I'm reading it wrong? But hey! What do I know? Its Valentine's Day and I'm spending my evening answering questions sitting in my man cave instead of spending it with my wife. Its no wonder I live in the perpetual dog house! :-P On Feb 14, 2013, at 7:35 PM, Rahul Ravindran rahu...@yahoo.com wrote: Checkandput() does not work when the row does not exist, or am I missing something? Sent from my phone.Excuse the terseness. On Feb 14, 2013, at 5:33 PM, Michael Segel michael_se...@hotmail.com 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 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
RE: Using HBase for Deduping
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.