hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Li Li <fancye...@gmail.com>
Subject Re: is this rowkey schema feasible?
Date Fri, 10 Jan 2014 02:08:38 GMT
thanks.
1. this is just a url frontier for url duplication and scheduler usage.
2. after get top N prior urls, I will send them to a message
queue(activemq) and the fetcher cluster will do their work.
3. you said keep the 'crawl status' in a separate column family, I
think it's a good idea. so update status will not delete a row and
then insert a new row.
4. you said Using an hbase table as a queue is usually not a good
idea, any other better solution for this? redis? memcached?I need a
distributed frontier which support order by(order by priority first
then enqueue time)
5. yes, I should use checkAndPut to avoid duplicated insert.
6. scan maybe not efficient?
    there are many urls,I don't know the status and priority of
existing url(the priority is calculated dynamically).because MD5(path)
is in the end of rowkey, I have to scan all the urls in the
host(www.google.com). Maybe scan all urls with the same
host is quicker than 20 get because the urls in a domain are likely
located in a single region(block). but 10M urls for a single host is
not uncommon.

    com.google.www/-0-10-MD5(path1)
    com.google.www/-0-10-MD5(path2)
    ...
    com.google.www/-0-10-MD5(path10,000,000)

    com.google.www/-1-10-MD5(path1)
    com.google.www/-1-10-MD5(path2)
    ....
    com.google.www/-1-10-MD5(path10,000,000)

    com.google.www/-0-9-MD5(path1)
    com.google.www/-0-9-MD5(path2)
    .....
    com.google.www/-1-1-MD5(path1)
    .....
    com.google.www/-1-1-MD5(path10,000,000)

On Fri, Jan 10, 2014 at 2:02 AM, Stack <stack@duboce.net> wrote:
> On Thu, Jan 9, 2014 at 2:42 AM, Li Li <fancyerii@gmail.com> wrote:
>
>> hi all,
>>     I want to use hbase to store all urls for a distributed crawler.
>> there is a central scheduler to schedule all unCrawled urls by
>> priority.
>
>
>
> Are you building from scratch?  If so, have you looked at nutch?
>
>
>
>> Following is my design of rowkey and common data access
>> pattern, is there any better rowkey design for my usecase?
>>
>>     the row key is: reverse_host--status--priority--MD5(path). some
>> example:
>>     com.google.www/-0-10-MD5(path1)
>>     com.google.www/-0-9-MD5(path2)
>>     ...
>>     com.google.www/-1-10-MD5(path3)
>>     status 0 means not crawled and 1 means crawled
>>
>
> Is this your total schema?  Where is crawl history kept and the crawled
> content and rate of change for the content and when to recrawl, etc?  Is
> this data elsewhere in other tables and this table is just a frontier
> 'queue' with crawl URLs added to the head and when the page is crawled, you
> add a new row to this table w/ state set to 1?  How you going to request a
> page be recrawled in such a scheme?
>
> How do the distributed crawlers divvy up the work?
>
> Generally you do not want to keep state in the key itself.
>
> Using an hbase table as a queue is usually not a good idea especially when
> lots of churn as will be the case in a distributed crawler.
>
> You could keep the 'crawl status' in a separate column family with nothing
> but this in it so your crawlers can scan fast and update this one attribute
> only after the page is pulled, or, you might want to use something else
> altogether for the list-of-urls to crawl by crawler since it a small
> dataset and you need to go real fast against it.
>
>
>
>
>
>>     my scheduler:
>>     int batchSize=10000;
>>     Map<String,Integer> hostCount=calcHostPriority(batchSize);
>>     List<String> toBeCrawledUrls=..
>>     for(Map.Entry<String,Integer> entry:hostCount.entrySet()){
>>          //select top N priority uncrawled urls for this host
>>         startRow=Bytes.toString(reverse(entry.getKey())+"/-0");
>>         stopRow=Bytes.toString(reverse(entry.getKey())+"/-1");
>>          Scan s = new Scan(startRow, stopRow);
>>          s.setMaxResultSize(entry.getValue());
>>          for(String url:scanResult){
>>               toBeCrawledUrls.add(url);
>>          }
>>     }
>>
>>     //update after crawling
>>     for(String url:crawledUrls){
>>          delete url //com.google.www/-0-10-MD5(path)
>>          put url //com.google.www/-1-10-MD5(path)
>>
>
> Each delete adds a new entry.
>
> How you intend to make it so two distributed crawlers do not pull the same
> url to fetch?  (Use checkAndPut and set a 'currently-assigned-to-a-crawler'
> flag).
>
>
>
>
>>     }
>>
>>     //check url exists
>>     any better method than this?
>>      assuming only 1-10 priority
>>    try get:
>>         com.google.www/-0-10-MD5(path)
>>         com.google.www/-1-10-MD5(path)
>>         com.google.www/-0-9-MD5(path)
>>         ....
>>         com.google.www/-1-1-MD5(path)
>>     if any exists, then true
>>     else false
>>
>
>
> You don't want to 'get', you want to 'scan'.
>
> St.Ack

Mime
View raw message