Performance of hbase importing

2009-01-11 Thread Ryan Rawson
Hi all,

New user of hbase here. I've been trolling about in IRC for a few days, and
been getting great help all around so far.

The topic turns to importing data into hbase - I have largeish datasets I
want to evaluate hbase performance on, so I've been working at importing
said data.  I've managed to get some impressive performance speedups, and I
chronicled them here:

http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html

To summarize:
- Use the Native HBASE API in Java or Jython (or presumably any JVM
language)
- Disable table auto flush, set write buffer large (12M for me)

At this point I can import a 18 GB, 440m row comma-seperated flat file in
about 72 minutes using map-reduce.  This is on a 3 node cluster all running
hdfs,hbase,mapred with 12 map tasks (4 per).  This hardware is loaner DB
hardware, so once I get my real cluster I'll revise/publish new data.

I look forward to meeting some of you next week at the hbase meetup at
powerset!

-ryan


Re: Question to speaker (tab file loading) at yesterdays user group

2009-01-15 Thread Ryan Rawson
I think you were referring to my presentation.

I was importing a CSV file, of 6 integers.  Obviously in the CSV file, the
integers were their ASCII representation.  So my code had to atoi() the
strings, then pack them into Thrift records, serialize those, and finally
insert the binary thrift rep into hbase with a key.

I had 3 versions:
- thrift gateway - this was the slowest, doing 20m records in 6 hours.  The
init code looks like:
transport = TSocket.TSocket(hbaseMaster, hbasePort)
transport = TTransport.TBufferedTransport(transport)
protocol = TBinaryProtocol.TBinaryProtocol(transport)
client = Hbase.Client(protocol)
transport.open()

So using buffered transport, but no specific hbase API calls to set auto
flush or other params. This is in CPython.

- HBase API version #1:
Written in Jython, this is substantially faster, doing 20m records in 70
minutes, or 4 per ms.  This performance scales up to at least 6 processes.

- HBase API version #2:
Slightly smarter, I now call:
table.setAutoFlush(False)
table.setWriteBufferSize(1024*1024*12)

And my speed jumps up to between 30-50 inserts per ms, scaling to at least 6
concurrent processes.

I then rewrote this stuff into a map-reduce and I can now insert 440m
records in about 70-80 minutes.

As I move forward, I will be emulating bigtable and using either thrift
serialized records or protobufs to store data in cells.  This allows you to
forward/backwards compatiblly extend data within individual cells.  Until
compression is super solid, I would be wary of storing text (xml, html, etc)
in hbase due to size concerns.


The hardware:
- 4 cpu, 128 gb ram
- 1 tb disk

Here are some relevant configs:
hbase-env.sh:
export HBASE_HEAPSIZE=5000

hadoop-site.xml:

dfs.datanode.socket.write.tiemout
0



dfs.datanode.max.xcievers
2047



dfs.datanode.handler.count
10







On Wed, Jan 14, 2009 at 11:30 PM, tim robertson
wrote:

> Hi all,
>
> I was skyping in yesterday from Europe.
> Being half asleep and on a bad wireless, it was not too easy to hear
> sometimes, and I have some quick questions to the person who was
> describing his tab file (CSV?) loading at the beginning.
>
> Could you please summarise quickly again the stats you mentioned?
> Number rows, size file size pre loading, was it 7 Strings? per row,
> size after load, time to load etc
>
> Also, could you please quickly summarise your cluster hardware (spec,
> ram + number nodes)?
>
> What did you find sped it up?
>
> How many columns per family were you using and did this affect much
> (presumably less mean fewer region splits right?)
>
> The reason I ask is I have around 50G in tab file (representing 162M
> rows from mysql with around 50 fields - strings of <20 chars and int
> mostly) and will be loading HBase with this.  Once this initial import
> is done, I will then harvest XML and Tab files into HBase directly
> (storing the raw XML record or tab file row as well).
> I am in early testing (awaiting hardware and fed up using EC2) so
> still running code on laptop and small tests.  I have 6 dell boxes (2
> proc, 5G memory, SCSI?) being freed up in 3-4 weeks and wonder what
> performance I will get.
>
> Thanks,
>
> Tim
>


Re: Question on cluster capacity planning

2009-01-15 Thread Ryan Rawson
I have found with my tests that 3 nodes is wholy insufficient.  I think it's
causing me to hit the xciever limit sooner than I would if I was running 10+
machines.  The issue is with r=3 on HDFS, and you have 3 machines, you get
reliability but no spreading of load.  I don't know how big the 'large EC2'
instances are, but you might want to consider running more of smaller for
the same cost if possible.  You get better spread of load across machines,
and should increase overall performance.

Also, how is it running on EC2?  What happens when your machines go away?
You have to rewrite and copy the config around, do you not?

One last thing, the master is very important, but also takes the least
load.  Running bigger iron for it seems pointless to me.  My master has a
load average of 0.00 at all times, including when I am running intense
import MR tasks that put a LA of 6+ on all my region server/datanode
servers.

-ryan

On Thu, Jan 15, 2009 at 3:05 AM, Michael Dagaev wrote:

> Hi, all
>
>How did you plan your Hbase cluster capacity ?
>
> Currently we run a cluster of 4 large EC2 instances
> (one master and 3 region servers). The throughput
> is Ok but the database is small now.
>
> Let's say we are preparing to store X terabytes.
> I guess the database size will impacts the performance.
> How many servers should we run in that case ?
>
> Thank you for your cooperation,
> M.
>


Re: Question to speaker (tab file loading) at yesterdays user group

2009-01-15 Thread Ryan Rawson
At the user meeting last night, stack noted that since lots of us "lads" are
noting performance improvement on random-read when we use compression, that
perhaps a fresh look at making compression solid would be a good thing.

Personally I am just obsessed with on-disk efficiency.  But also, I am
chasing after random-read performance latencys so I can serve a website out
of hbase... if that isnt your needs, then perhaps what you want to do would
be just fine as it?

-ryan

On Thu, Jan 15, 2009 at 2:11 PM, tim robertson wrote:

> > Until compression is super solid, I would be wary of storing text (xml,
> html, etc)
> > in hbase due to size concerns.
>
> Hmmm... Where do the indexing guys store their raw harvested records /
> HTML / whatever then?
>
> I guess mine would be coming in at 200G as text or so, per 100M
> records (maybe looking to 1Billion records over next 24 months).  Can
> someone suggest a better place to store the records if not HBase?  I
> want to be able to serve them as cached records, and also use them as
> sources for new indexes, without harvesting again.  This is classic
> use case of HBase I thought... I mean, it is even on the HBase
> architecture page as the example table structure:
> http://wiki.apache.org/hadoop/Hbase/HbaseArchitecture.  Bit surprised
> to hear it is not recommended use.
>
> Cheers for pointers and sorry for the question bombardment - just
> trying to catch up.
>
> Tim
>
>
>
>
>
>
>
> On Thu, Jan 15, 2009 at 10:12 PM, Ryan Rawson  wrote:
> > I think you were referring to my presentation.
> >
> > I was importing a CSV file, of 6 integers.  Obviously in the CSV file,
> the
> > integers were their ASCII representation.  So my code had to atoi() the
> > strings, then pack them into Thrift records, serialize those, and finally
> > insert the binary thrift rep into hbase with a key.
> >
> > I had 3 versions:
> > - thrift gateway - this was the slowest, doing 20m records in 6 hours.
>  The
> > init code looks like:
> >transport = TSocket.TSocket(hbaseMaster, hbasePort)
> >transport = TTransport.TBufferedTransport(transport)
> >protocol = TBinaryProtocol.TBinaryProtocol(transport)
> >client = Hbase.Client(protocol)
> >transport.open()
> >
> > So using buffered transport, but no specific hbase API calls to set auto
> > flush or other params. This is in CPython.
> >
> > - HBase API version #1:
> > Written in Jython, this is substantially faster, doing 20m records in 70
> > minutes, or 4 per ms.  This performance scales up to at least 6
> processes.
> >
> > - HBase API version #2:
> > Slightly smarter, I now call:
> > table.setAutoFlush(False)
> > table.setWriteBufferSize(1024*1024*12)
> >
> > And my speed jumps up to between 30-50 inserts per ms, scaling to at
> least 6
> > concurrent processes.
> >
> > I then rewrote this stuff into a map-reduce and I can now insert 440m
> > records in about 70-80 minutes.
> >
> > As I move forward, I will be emulating bigtable and using either thrift
> > serialized records or protobufs to store data in cells.  This allows you
> to
> > forward/backwards compatiblly extend data within individual cells.  Until
> > compression is super solid, I would be wary of storing text (xml, html,
> etc)
> > in hbase due to size concerns.
> >
> >
> > The hardware:
> > - 4 cpu, 128 gb ram
> > - 1 tb disk
> >
> > Here are some relevant configs:
> > hbase-env.sh:
> > export HBASE_HEAPSIZE=5000
> >
> > hadoop-site.xml:
> > 
> > dfs.datanode.socket.write.tiemout
> > 0
> > 
> >
> > 
> > dfs.datanode.max.xcievers
> > 2047
> > 
> >
> > 
> > dfs.datanode.handler.count
> > 10
> > 
> >
> >
> >
> >
> >
> >
> > On Wed, Jan 14, 2009 at 11:30 PM, tim robertson
> > wrote:
> >
> >> Hi all,
> >>
> >> I was skyping in yesterday from Europe.
> >> Being half asleep and on a bad wireless, it was not too easy to hear
> >> sometimes, and I have some quick questions to the person who was
> >> describing his tab file (CSV?) loading at the beginning.
> >>
> >> Could you please summarise quickly again the stats you mentioned?
> >> Number rows, size file size pre loading, was it 7 Strings? per row,
> >> size after load, time to load etc
> >>
> >> Also, could you please quickly summarise your cluster hardware (spec,
> >> ram + number nodes)?
> >>
> >> What did you find sped it up?
&

Re: Another Question on Backup Tables

2009-01-23 Thread Ryan Rawson
You'd have to get at least:

/hbase/$TABLE_NAME
and
/hbase/log*

Not too sure how hbase would handle that though...  The logs contain
roll-forward info for every table.  So you'd have entries for tables that
dont exist.

-ryan

On Fri, Jan 23, 2009 at 2:05 AM,  wrote:

> Hi, all
>
> As I understand from the answers of Jean-Daniel and Stack
> we can backup the database just by copying the HDFS folder
> but what if I want to backup only a few tables ?
>
> I guess I can scan the tables and copy the scanned data
> to somewhere on the backup storage. Are there other solutions ?
>
> Thank you for your cooperation,
> M.
>


Re: Another Question on Backup Tables

2009-01-23 Thread Ryan Rawson
At some point it would be nice if we could get a hbase replication module.
Maybe later this year?

-ryan

On Fri, Jan 23, 2009 at 5:38 PM, Erik Holstad  wrote:

> Michael!
> The code for 974 should be up now and ready for you to test.
> Please let us know if you have any problems.
>
> Erik
>
> On Fri, Jan 23, 2009 at 10:51 AM, stack  wrote:
>
> > You'd also need to copy out the tables' entries in .META.
> >
> > There are some scripts in HBASE-643 for copying and renaming tables that
> > might serve as starting point copying a table.
> >
> > St.Ack
> >
> >
> > Ryan Rawson wrote:
> >
> >> You'd have to get at least:
> >>
> >> /hbase/$TABLE_NAME
> >> and
> >> /hbase/log*
> >>
> >> Not too sure how hbase would handle that though...  The logs contain
> >> roll-forward info for every table.  So you'd have entries for tables
> that
> >> dont exist.
> >>
> >> -ryan
> >>
> >> On Fri, Jan 23, 2009 at 2:05 AM,  wrote:
> >>
> >>
> >>
> >>> Hi, all
> >>>
> >>> As I understand from the answers of Jean-Daniel and Stack
> >>> we can backup the database just by copying the HDFS folder
> >>> but what if I want to backup only a few tables ?
> >>>
> >>> I guess I can scan the tables and copy the scanned data
> >>> to somewhere on the backup storage. Are there other solutions ?
> >>>
> >>> Thank you for your cooperation,
> >>> M.
> >>>
> >>>
> >>>
> >>
> >>
> >>
> >
> >
>


Re: HBase features

2009-01-24 Thread Ryan Rawson
One thing about bigtable (and therefore hbase) is that the system doesn't
provide much in the way for support of #3.  No declarative indexes - you can
have them, but you have to keep them up to date with code.  No sorting, no
distributed functions, nothing of the sort.  Do it in client code, use a
map-reduce for a longer computation.

But bigtable/hbase lets you craft your row-keys and column families to
achieve high locality of reference for associated data.  Read the bigtable
paper for info, pay attention to the way the have keys for webpages.

It's very simple, but can you scale a RMDBS to 1000 TB + ?

-ryan

On Sat, Jan 24, 2009 at 8:44 PM, Adam Rose  wrote:

> Hi folks -
>
> New to the community here, but encouraging to see it's alive and active.
>  :)
>
> We're currently assessing HBase as an option to provide tables that are
> queried directly from a web interface, for an analytics application.  I'm
> wondering if anyone has an example of how a typical analytic data warehouse
> star and/or snowflake schema design might look in HBase?  This stuff is
> extremely well documented on how to do in a traditional RDBMS, but I can't
> find a simple example of this type of schema design.
>
> As well, is there a summary of how HBase may compare to BigTable, in terms
> of features.?  Obviously BigTable is closed-source, so we can't know
> definitively.  But I've been looking at
> http://code.google.com/appengine/docs/python/datastore/, and am wondering
> if there obvious features that HBase is missing.?  Specifically, I'm
> wondering if BigTable and HBase compare in the areas of:
>
> 1. What is done client-side vs. in the data store / BigTable
> 2. (Distributed?) query support with filters
> 3. Indexing, (distributed?) sorting, distributed SUM, AVG, etc.
> 4. Parallelization of queries and writes
> 5. Anything else relevant
>
> Any pointers or help much appreciated.
>
> Thx - Adam
>
>


Re: writing map output to local fileSystem

2009-02-03 Thread Ryan Rawson
Hi there,

This is the hbase user list. Hbase is a google bigtable inspired project.
You want the hadoop-user list. Try hadoop's website.

Good luck!

On Feb 3, 2009 2:42 AM, "Yair Even-Zohar"  wrote:

I'm trying to run a map and output data to local filesystem on EC2 and
run into some problems.

prior to hadoop/hbase 19 I was using



   RawLocalFileSystem rlfs = new RawLocalFileSystem();

   Path path = new Path("file:///directory");

   rlfs.setWorkingDirectory(path);

   FileOutputFormat.setOutputPath(c, rlfs.getWorkingDirectory());



All I'm getting now under the blah directory is just the _logs directory
and the other output files are not there.

If I write to HDFS all the required data is in place.



Any idea?



Thanks

-Yair


Re: Hbase Exceptions

2009-02-03 Thread Ryan Rawson
Try upping your xcievers to 2047 or thereabouts.  I had to do that with a
cluster of your size.

Was there any errors on the datanode side you could find?

-ryan

On Tue, Feb 3, 2009 at 1:58 AM, Michael Dagaev wrote:

> Hi, all
>
> We ran an HBase cluster of (1 master/name node + 3 region
> server/data nodes).
> We upped the number of open files per process, increased the heap size
> of the region
> servers and data nodes to 2G, and set dfs.datanode.socket.write.timeout=0,
> and
> dfs.datanode.max.xcievers=1023
>
> The cluster seems to run ok but the Hbase logged exceptions at INFO/DEBUG
> level.
> For instance
>
>org.apache.hadoop.dfs.DFSClient: Could not obtain block 
>from any node:  java.io.IOException: No live nodes contain current block
>
>   org.apache.hadoop.dfs.DFSClient: Failed to connect to :50010:
>   java.io.IOException: Got error in response to OP_READ_BLOCK for
> file 
>
> Does anybody know what these exceptions mean and how to fix them?
>
> Thank you for your cooperation,
> M.
>


Re: writing map output to local fileSystem

2009-02-03 Thread Ryan Rawson
You got it - you can't "update" the row key without a
reading/updating/delete/insert cycle.  This is because the newly inserted
row might not live on the same region server anymore.

It would probably be better to have a schema that avoids needing it's
primary key updated.

One common design pattern is to use a system like protobufs or thrift
serialization to store structured binary data in the hbase cells, thus
upping the complexity of what you may store in a hbase row. With some clever
redesign you may discover you can avoid the primary key value update.

Good luck!
-ryan

On Tue, Feb 3, 2009 at 6:23 AM, Yabo-Arber Xu wrote:

> Hi there,
>
> I have this usage scenario on HBase, and wonder what is the most efficient
> way of doing this:
>
> I use each row to represent some cluster, and the rowkey is sort of the
> center of the cluster. So every time I add an element into a cluster, i
> need
> to update the rowkey ( also some minor additional updates to certain
> columns).
>
> The best way I know is to read this whole row out, remove it from hbase,
> and
> insert the same row with new rowkey, but this appears not to be that
> efficient. Any thoughts?
>
> Thanks for your input!
>
> Best,
> Arber
>


Re: Hbase first run problem

2009-02-11 Thread Ryan Rawson
Do:
hadoop fs -rmr /hbase

and try again.

HBase will auto-create the /hbase dir on the first run.

-ryan

On Wed, Feb 11, 2009 at 1:50 AM, Sergey Bartunov  wrote:

> Hi. I'm trying to make hbase (0.19) running on my ubuntu 8.10. Hadoop
> (0.19) already works in single-node mode:
>
> 
>  
>fs.default.name
>hdfs://localhost:9000
>  
>  
>mapred.job.tracker
>localhost:9001
>  
>  
>dfs.replication
>1
>  
> 
>
> I've formatted namenode and created /hbase directory, which is
> specified in hbase-site.xml:
>
> 
>  
> hbase.master
> localhost:6
> The host and port that the HBase master runs
> at
>   
>
>   
> hbase.rootdir
> hdfs://localhost:9000/hbase
> location of HBase instance in dfs
>   
>
> 
>
> When I try to hbase/bin/start-hbase.sh HMaster falls with warning:
>
> WARNING! File system needs to be upgraded. Run the
> '${HBASE_HOME}/bin/hbase migrate' script.
>
> But I can't upgrade because file hbase.version doesn't exist and I
> don't understand why I need to upgrade if I use the last version of
> hadoop. Also I have no idea how to get hbase.version file.
> What I have to do to make this work? Thanks in advance
>
> --
> Rideamus!
>


Re: Extra java.library.path config parameter need?

2009-02-19 Thread Ryan Rawson
If you start jython with a CLASSPATH it will respect said classpath...

That is how I got my jython + hbase work done.

-ryan

On Thu, Feb 19, 2009 at 5:46 PM, xiaohui zhang  wrote:

> Hi, everybody,
> I want to use HBase with Jython and followed the instructions of
> http://wiki.apache.org/hadoop/Hbase/Jython. Every thing goes fine, but the
> command line editor. I googled and find out that Jython need a native edit
> line library(http://wiki.python.org/jython/ReadlineSetup) to work
> correctly.
>
> It seems there is no way to set a extra java.library.path for HBase. Do I
> have to copy the library to HBase's native library path or manually modify
> the hbase script?
>
> Thanks.
>
> Best regards,
> Xiaohui
>


Re: Connection problem during data import into hbase

2009-02-21 Thread Ryan Rawson
i run in to that a lot - disabling a table doesnt seem to work all the time.

i think the zk support in 0.20 will help fix many of these 'cant find
regionserver' and other sync issues.

On Sat, Feb 21, 2009 at 1:44 AM, Amandeep Khurana  wrote:

> Here's another thing thats happening. I was trying to truncate the table.
>
> hbase(main):001:0> truncate 'in_table'
> Truncating in_table; it may take a while
> Disabling table...
> NativeException: org.apache.hadoop.hbase.RegionException: Retries
> exhausted,
> it took too long to wait for the table in_table to be disabled.
>from org/apache/hadoop/hbase/client/HBaseAdmin.java:387:in
> `disableTable'
>from org/apache/hadoop/hbase/client/HBaseAdmin.java:348:in
> `disableTable'
> from sun.reflect.NativeMethodAccessorImpl:-2:in `invoke0'
>from sun.reflect.NativeMethodAccessorImpl:-1:in `invoke'
>from sun.reflect.DelegatingMethodAccessorImpl:-1:in `invoke'
>from java.lang.reflect.Method:-1:in `invoke'
>from org/jruby/javasupport/JavaMethod.java:250:in
> `invokeWithExceptionHandling'
>from org/jruby/javasupport/JavaMethod.java:219:in `invoke'
>from org/jruby/javasupport/JavaClass.java:416:in `execute'
> from org/jruby/internal/runtime/methods/SimpleCallbackMethod.java:67:in
> `call'
>from org/jruby/internal/runtime/methods/DynamicMethod.java:78:in `call'
>from org/jruby/runtime/CallSite.java:155:in `cacheAndCall'
>from org/jruby/runtime/CallSite.java:332:in `call'
>from org/jruby/evaluator/ASTInterpreter.java:649:in `callNode'
>from org/jruby/evaluator/ASTInterpreter.java:324:in `evalInternal'
>
> I left it for a few minutes and tried again. It worked. There was no load
> on
> the cluster at all. changed the config (both) and added
> dfs.datanode.socket.write.timeout property with value 0. I also defined the
> property in the job config.
>
> Amandeep
>
>
> Amandeep Khurana
> Computer Science Graduate Student
> University of California, Santa Cruz
>
>
> On Sat, Feb 21, 2009 at 1:23 AM, Amandeep Khurana 
> wrote:
>
> > I have 1 master + 2 slaves.
> > Am using 0.19.0 for both Hadoop and Hbase.
> > I didnt change any config from the default except the hbase.rootdir and
> the
> > hbase.master.
> >
> > I have gone through the FAQs but couldnt find anything. What exactly are
> > you pointing to?
> >
> >
> > Amandeep Khurana
> > Computer Science Graduate Student
> > University of California, Santa Cruz
> >
> >
> > On Sat, Feb 21, 2009 at 1:14 AM, stack  wrote:
> >
> >> It looks like regionserver hosting root crashed:
> >>
> >> org.apache.hadoop.hbase.client.NoServerForRegionException: Timed out
> >> trying
> >> to locate root region
> >>
> >> How many servers you running?
> >>
> >> You made similar config. to that reported by Larry Compton in a mail
> from
> >> earlier today?  (See FAQ and Troubleshooting page for more on his listed
> >> configs.)
> >>
> >> St.Ack
> >>
> >>
> >> On Sat, Feb 21, 2009 at 1:01 AM, Amandeep Khurana 
> >> wrote:
> >>
> >> > Yes, the table exists before I start the job.
> >> >
> >> > I am not using TableOutputFormat. I picked up the sample code from the
> >> docs
> >> > and am using it.
> >> >
> >> > Here's the job conf:
> >> >
> >> > JobConf conf = new JobConf(getConf(), IN_TABLE_IMPORT.class);
> >> >FileInputFormat.setInputPaths(conf, new Path("import_data"));
> >> >conf.setMapperClass(MapClass.class);
> >> >conf.setNumReduceTasks(0);
> >> >conf.setOutputFormat(NullOutputFormat.class);
> >> >JobClient.runJob(conf);
> >> >
> >> > Interestingly, the hbase shell isnt working now either. Its giving
> >> errors
> >> > even when I give the command "list"...
> >> >
> >> >
> >> >
> >> > Amandeep Khurana
> >> > Computer Science Graduate Student
> >> > University of California, Santa Cruz
> >> >
> >> >
> >> > On Sat, Feb 21, 2009 at 12:10 AM, stack  wrote:
> >> >
> >> > > The table exists before you start the MR job?
> >> > >
> >> > > When you say 'midway through the job', are you using
> tableoutputformat
> >> to
> >> > > insert into your table?
> >> > >
> >> > > Which version of hbase?
> >> > >
> >> > > St.Ack
> >> > >
> >> > > On Fri, Feb 20, 2009 at 9:55 PM, Amandeep Khurana  >
> >> > > wrote:
> >> > >
> >> > > > I dont know if this is related or not, but it seems to be. After
> >> this
> >> > map
> >> > > > reduce job, I tried to count the number of entries in the table in
> >> > hbase
> >> > > > through the shell. It failed with the following error:
> >> > > >
> >> > > > hbase(main):002:0> count 'in_table'
> >> > > > NativeException: java.lang.NullPointerException: null
> >> > > >from java.lang.String:-1:in `'
> >> > > >from org/apache/hadoop/hbase/util/Bytes.java:92:in `toString'
> >> > > >from
> >> > > org/apache/hadoop/hbase/client/RetriesExhaustedException.java:50:in
> >> > > > `getMessage'
> >> > > >from
> >> > > org/apache/hadoop/hbase/client/RetriesExhaustedException.java:40:in
> >> > > > `'
> >> > > >from
> >> org/apache/had

Re: Connection problem during data import into hbase

2009-02-21 Thread Ryan Rawson
So the usual suspects are:

- xcievers (i hvae mine set to 2047)
- timeout (i have mine set to 0)

I can import a few hundred million records with these settings.

how many nodes do you have again?

On Sat, Feb 21, 2009 at 2:14 AM, Amandeep Khurana  wrote:

> Yes, I noticed it this time. The regionserver gets slow or stops responding
> and then this error comes. How do I get this to work? Is there a way of
> limiting the resources that the map red job should take?
>
> I did make the changes in the config site similar to Larry Comton's config.
> It only made the job go from dying at 7% to 12% this time.
>
> Amandeep
>
>
> Amandeep Khurana
> Computer Science Graduate Student
> University of California, Santa Cruz
>
>
> On Sat, Feb 21, 2009 at 1:14 AM, stack  wrote:
>
> > It looks like regionserver hosting root crashed:
> >
> > org.apache.hadoop.hbase.client.NoServerForRegionException: Timed out
> trying
> > to locate root region
> >
> > How many servers you running?
> >
> > You made similar config. to that reported by Larry Compton in a mail from
> > earlier today?  (See FAQ and Troubleshooting page for more on his listed
> > configs.)
> >
> > St.Ack
> >
> >
> > On Sat, Feb 21, 2009 at 1:01 AM, Amandeep Khurana 
> > wrote:
> >
> > > Yes, the table exists before I start the job.
> > >
> > > I am not using TableOutputFormat. I picked up the sample code from the
> > docs
> > > and am using it.
> > >
> > > Here's the job conf:
> > >
> > > JobConf conf = new JobConf(getConf(), IN_TABLE_IMPORT.class);
> > >FileInputFormat.setInputPaths(conf, new Path("import_data"));
> > >conf.setMapperClass(MapClass.class);
> > >conf.setNumReduceTasks(0);
> > >conf.setOutputFormat(NullOutputFormat.class);
> > >JobClient.runJob(conf);
> > >
> > > Interestingly, the hbase shell isnt working now either. Its giving
> errors
> > > even when I give the command "list"...
> > >
> > >
> > >
> > > Amandeep Khurana
> > > Computer Science Graduate Student
> > > University of California, Santa Cruz
> > >
> > >
> > > On Sat, Feb 21, 2009 at 12:10 AM, stack  wrote:
> > >
> > > > The table exists before you start the MR job?
> > > >
> > > > When you say 'midway through the job', are you using
> tableoutputformat
> > to
> > > > insert into your table?
> > > >
> > > > Which version of hbase?
> > > >
> > > > St.Ack
> > > >
> > > > On Fri, Feb 20, 2009 at 9:55 PM, Amandeep Khurana 
> > > > wrote:
> > > >
> > > > > I dont know if this is related or not, but it seems to be. After
> this
> > > map
> > > > > reduce job, I tried to count the number of entries in the table in
> > > hbase
> > > > > through the shell. It failed with the following error:
> > > > >
> > > > > hbase(main):002:0> count 'in_table'
> > > > > NativeException: java.lang.NullPointerException: null
> > > > >from java.lang.String:-1:in `'
> > > > >from org/apache/hadoop/hbase/util/Bytes.java:92:in `toString'
> > > > >from
> > > > org/apache/hadoop/hbase/client/RetriesExhaustedException.java:50:in
> > > > > `getMessage'
> > > > >from
> > > > org/apache/hadoop/hbase/client/RetriesExhaustedException.java:40:in
> > > > > `'
> > > > >from
> org/apache/hadoop/hbase/client/HConnectionManager.java:841:in
> > > > > `getRegionServerWithRetries'
> > > > >from org/apache/hadoop/hbase/client/MetaScanner.java:56:in
> > > `metaScan'
> > > > >from org/apache/hadoop/hbase/client/MetaScanner.java:30:in
> > > `metaScan'
> > > > >from
> org/apache/hadoop/hbase/client/HConnectionManager.java:411:in
> > > > > `getHTableDescriptor'
> > > > >from org/apache/hadoop/hbase/client/HTable.java:219:in
> > > > > `getTableDescriptor'
> > > > >from sun.reflect.NativeMethodAccessorImpl:-2:in `invoke0'
> > > > >from sun.reflect.NativeMethodAccessorImpl:-1:in `invoke'
> > > > >from sun.reflect.DelegatingMethodAccessorImpl:-1:in `invoke'
> > > > >from java.lang.reflect.Method:-1:in `invoke'
> > > > >from org/jruby/javasupport/JavaMethod.java:250:in
> > > > > `invokeWithExceptionHandling'
> > > > >from org/jruby/javasupport/JavaMethod.java:219:in `invoke'
> > > > >from org/jruby/javasupport/JavaClass.java:416:in `execute'
> > > > > ... 145 levels...
> > > > >from org/jruby/internal/runtime/methods/DynamicMethod.java:74:in
> > > > `call'
> > > > >from
> org/jruby/internal/runtime/methods/CompiledMethod.java:48:in
> > > > `call'
> > > > >from org/jruby/runtime/CallSite.java:123:in `cacheAndCall'
> > > > >from org/jruby/runtime/CallSite.java:298:in `call'
> > > > >from
> > > > >
> > > > >
> > > >
> > >
> >
> ruby/hadoop/install/hbase_minus_0_dot_19_dot_0/bin//hadoop/install/hbase/bin/../bin/hirb.rb:429:in
> > > > > `__file__'
> > > > >from
> > > > >
> > > > >
> > > >
> > >
> >
> ruby/hadoop/install/hbase_minus_0_dot_19_dot_0/bin//hadoop/install/hbase/bin/../bin/hirb.rb:-1:in
> > > > > `__file__'
> > > > >from
> > > > >
> > > > >
> > > >
> > >
> >
> ruby/hadoop/install/hbase_minus_0_dot_19_dot_0/b

Re: Connection problem during data import into hbase

2009-02-21 Thread Ryan Rawson
you have to change hadoop-site.xml and restart HDFS.

you should also change the logging to be more verbose in hbase - check out
the hbase FAQ (link missing -ed).

if you get the problem again, peruse the hbase logs and post what is going
on there.  the client errors dont really include the root cause on the
regionserver side.

good luck,
-ryan


On Sat, Feb 21, 2009 at 2:21 AM, Amandeep Khurana  wrote:

> I have 1 master + 2 slaves. I did set the timout to zero. I'll set the
> xceivers to 2047 and try again. Can this be done in the job config or does
> the site.xml need to be changed and the cluster restarted?
>
> Amandeep
>
>
> Amandeep Khurana
> Computer Science Graduate Student
> University of California, Santa Cruz
>
>
> On Sat, Feb 21, 2009 at 2:16 AM, Ryan Rawson  wrote:
>
> > So the usual suspects are:
> >
> > - xcievers (i hvae mine set to 2047)
> > - timeout (i have mine set to 0)
> >
> > I can import a few hundred million records with these settings.
> >
> > how many nodes do you have again?
> >
> > On Sat, Feb 21, 2009 at 2:14 AM, Amandeep Khurana 
> > wrote:
> >
> > > Yes, I noticed it this time. The regionserver gets slow or stops
> > responding
> > > and then this error comes. How do I get this to work? Is there a way of
> > > limiting the resources that the map red job should take?
> > >
> > > I did make the changes in the config site similar to Larry Comton's
> > config.
> > > It only made the job go from dying at 7% to 12% this time.
> > >
> > > Amandeep
> > >
> > >
> > > Amandeep Khurana
> > > Computer Science Graduate Student
> > > University of California, Santa Cruz
> > >
> > >
> > > On Sat, Feb 21, 2009 at 1:14 AM, stack  wrote:
> > >
> > > > It looks like regionserver hosting root crashed:
> > > >
> > > > org.apache.hadoop.hbase.client.NoServerForRegionException: Timed out
> > > trying
> > > > to locate root region
> > > >
> > > > How many servers you running?
> > > >
> > > > You made similar config. to that reported by Larry Compton in a mail
> > from
> > > > earlier today?  (See FAQ and Troubleshooting page for more on his
> > listed
> > > > configs.)
> > > >
> > > > St.Ack
> > > >
> > > >
> > > > On Sat, Feb 21, 2009 at 1:01 AM, Amandeep Khurana 
> > > > wrote:
> > > >
> > > > > Yes, the table exists before I start the job.
> > > > >
> > > > > I am not using TableOutputFormat. I picked up the sample code from
> > the
> > > > docs
> > > > > and am using it.
> > > > >
> > > > > Here's the job conf:
> > > > >
> > > > > JobConf conf = new JobConf(getConf(), IN_TABLE_IMPORT.class);
> > > > >FileInputFormat.setInputPaths(conf, new
> Path("import_data"));
> > > > >conf.setMapperClass(MapClass.class);
> > > > >conf.setNumReduceTasks(0);
> > > > >conf.setOutputFormat(NullOutputFormat.class);
> > > > >JobClient.runJob(conf);
> > > > >
> > > > > Interestingly, the hbase shell isnt working now either. Its giving
> > > errors
> > > > > even when I give the command "list"...
> > > > >
> > > > >
> > > > >
> > > > > Amandeep Khurana
> > > > > Computer Science Graduate Student
> > > > > University of California, Santa Cruz
> > > > >
> > > > >
> > > > > On Sat, Feb 21, 2009 at 12:10 AM, stack  wrote:
> > > > >
> > > > > > The table exists before you start the MR job?
> > > > > >
> > > > > > When you say 'midway through the job', are you using
> > > tableoutputformat
> > > > to
> > > > > > insert into your table?
> > > > > >
> > > > > > Which version of hbase?
> > > > > >
> > > > > > St.Ack
> > > > > >
> > > > > > On Fri, Feb 20, 2009 at 9:55 PM, Amandeep Khurana <
> > ama...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > I dont know if this is related or not, but it seems to be.
> After
> > > this
>

Re: HBase and Web-Scale BI

2009-02-25 Thread Ryan Rawson
Hey,

You have to be clear about what hbase does and does not do.  HBase is just
not a rational database - it's "weakness" is it's strength.

In general, you can only access rows in key order.  Keys are stored
lexicographically sorted however.  There aren't declarative secondary
indexes (minus the lucene thing, but that isn't an index).  You have to put
all these pieces together to build a system.

But, you get scalability, and reasonable performance, and in 0.20 you will
get really good performance (fast enough to serve websites hopefully).

In general you need to make sure your row-key sorts data in the order you
want to query by.  You can do something like this:

  

to store events in reverse chronological order by users.

If you want another access method, you need to use a map-reduce and build a
secondary index.

I dont know if this exactly answers your question, but hopefully should give
you more of an idea of what hbase does and does not do.

-ryan





On Wed, Feb 25, 2009 at 9:02 PM, Bradford Stephens <
bradfordsteph...@gmail.com> wrote:

> Greetings,
>
> I'm in charge of the data analysis and collection platform at my company,
> and we're basing a large part of our core analysis platform on Hadoop,
> Nutch, and Lucene -- it's a delight to use. However, we're going to be
> wanting some on-demand "web-scale" business intelligence, and I'm wondering
> if HBase is the right solution -- my research hasn't given me any
> conclusions.
>
> Our data set is pretty simple -- a bunch of XML documents which have been
> parsed from HTML pages, and some associated data (Author Name, Post Date,
> Influence, etc). What we would like to be able to do is have our end users
> do real-time (< 10 seconds) OLAP-type analysis on this, and have it
> presented on a webpage. For example, queries like ("All authors for the
> past
> two weeks who have used these keywords in the post bodies and what their
> influence score is"). I imagine we'll have several terabytes of data to go
> through, and we won't be able to do much pre-population of results.
>
> Is HBase low-latency enough that we can scale-out to solve these sorts of
> problems?
>
> Cheers,
> Bradford
>


Re: HBase and Web-Scale BI

2009-02-26 Thread Ryan Rawson
I may have misspoke somewhat - hbase is actually quite good at random
reads.  But the catch is, it can only randomly read via the row id.  It's
more or less akin to having a DB table with only a index primary key, and no
secondary indexes.

So, yes, random reads and "index scans" work, and work well.  You just have
to handle the index creation and maintenance yourself.

-ryan

On Thu, Feb 26, 2009 at 12:06 PM, Jonathan Gray  wrote:

> Bradford,
>
> Many of us probably have some input but it's really difficult to help
> without having more detail.
>
> Can you be more specific about the layout of the data and the queries you'd
> want to run?
>
> HBase is efficient at scanning (as with hdfs), but also efficient at
> randomly accessing by row key.  If you need to fetch based on column names
> or values, then hbase will not be efficient without some form of secondary
> indexing (additional tables in hbase or something external like lucene).
>
> JG
>
> > -Original Message-
> > From: Bradford Stephens [mailto:bradfordsteph...@gmail.com]
> > Sent: Thursday, February 26, 2009 10:37 AM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: HBase and Web-Scale BI
> >
> > Yes, it seems that the fundamental 'differentness' of HDFS/MapReduce is
> > that
> > they're not very well suited to random access -- I was hoping HBase had
> > found a way 'around' that, but of course that 'differentness' is a
> > fundamental strength of the HDFS way of doing things.
> >
> > Where things have gotten murky is that our data is very simple -- we
> > just
> > have a lot of it. And we don't need to do a *lot* of random access to
> > our
> > data -- it really doesn't feel like an RDBMS situation.
> >
> > Perhaps if we made an index out of a hash of each of our data values,
> > and
> > did some 'normalization',  that could be the key. Or maybe the metadata
> > is
> > not going to be as large as I thought... hrm.
> >
> > I appreciate the input, and hope more people will chime in :)
> >
> > On Wed, Feb 25, 2009 at 10:18 PM, Ryan Rawson 
> > wrote:
> >
> > > Hey,
> > >
> > > You have to be clear about what hbase does and does not do.  HBase is
> > just
> > > not a rational database - it's "weakness" is it's strength.
> > >
> > > In general, you can only access rows in key order.  Keys are stored
> > > lexicographically sorted however.  There aren't declarative secondary
> > > indexes (minus the lucene thing, but that isn't an index).  You have
> > to put
> > > all these pieces together to build a system.
> > >
> > > But, you get scalability, and reasonable performance, and in 0.20 you
> > will
> > > get really good performance (fast enough to serve websites
> > hopefully).
> > >
> > > In general you need to make sure your row-key sorts data in the order
> > you
> > > want to query by.  You can do something like this:
> > >
> > >   
> > >
> > > to store events in reverse chronological order by users.
> > >
> > > If you want another access method, you need to use a map-reduce and
> > build a
> > > secondary index.
> > >
> > > I dont know if this exactly answers your question, but hopefully
> > should
> > > give
> > > you more of an idea of what hbase does and does not do.
> > >
> > > -ryan
> > >
> > >
> > >
> > >
> > >
> > > On Wed, Feb 25, 2009 at 9:02 PM, Bradford Stephens <
> > > bradfordsteph...@gmail.com> wrote:
> > >
> > > > Greetings,
> > > >
> > > > I'm in charge of the data analysis and collection platform at my
> > company,
> > > > and we're basing a large part of our core analysis platform on
> > Hadoop,
> > > > Nutch, and Lucene -- it's a delight to use. However, we're going to
> > be
> > > > wanting some on-demand "web-scale" business intelligence, and I'm
> > > wondering
> > > > if HBase is the right solution -- my research hasn't given me any
> > > > conclusions.
> > > >
> > > > Our data set is pretty simple -- a bunch of XML documents which
> > have been
> > > > parsed from HTML pages, and some associated data (Author Name, Post
> > Date,
> > > > Influence, etc). What we would like to be able to do is have our
> > end
> > > users
> > > > do real-time (< 10 seconds) OLAP-type analysis on this, and have it
> > > > presented on a webpage. For example, queries like ("All authors for
> > the
> > > > past
> > > > two weeks who have used these keywords in the post bodies and what
> > their
> > > > influence score is"). I imagine we'll have several terabytes of
> > data to
> > > go
> > > > through, and we won't be able to do much pre-population of results.
> > > >
> > > > Is HBase low-latency enough that we can scale-out to solve these
> > sorts of
> > > > problems?
> > > >
> > > > Cheers,
> > > > Bradford
> > > >
> > >
>
>


Re: Data uniqueness

2009-03-05 Thread Ryan Rawson
The only method of determining uniqueness of data in general in hbase is via
the row key.  Just like a primary key in a database, you can use it to
verify uniqueness, and do index scans and gets.

So generally speaking, yes you will have to make multiple trips to the
server to use a secondary index.  The situation might not be as dire as it
seems, since in 0.20 the speed targets for small data gets/sets is really
low (like maybe 1 ms?).

The solution to "need to do more" for hbase is generally 'well use
map-reduce'... which is the solution i will offer you as well.

Hopefully this answers some of your questions.

Good luck!
-ryan

On Thu, Mar 5, 2009 at 1:00 AM, Eran Bergman wrote:

> Hello,
>
> Lately I have been experimenting with HBase and I came across a problem I
> don't know how to solve yet.
> My problem is data uniqueness, meaning I would like to have unique data in
> a
> specified column (taking into account all or some subset of my rows).
> I would like to have that for any number of columns which I will specify
> (various types of data).
>
> Usually the way to do this is to use some sort of indexing method, but this
> will amount to round trips to the server for uniqueness checks before I
> commit, which are very costly.
>
> Does anyone have any thoughts on how to do this?
>
>
> Thanks,
> Eran
>


is -javaagent considered harmful?

2009-03-07 Thread Ryan Rawson
Hi all,

Well it turns out that Java does have a "sizeof"... you can access it via an
Java agent which gets a handle into java.lang.instrument.  But this requires
you to run a Java Agent JAR.

This is clearly suitable for experiments or profiling, but would it be
possible to run this full-time without performance problems?

Thanks,
-ryan


Re: Many columns in 0.19

2009-03-09 Thread Ryan Rawson
Sadly this is still a limit.

0.20 should make things much better.

-ryan

On Mon, Mar 9, 2009 at 12:23 AM, Michael Dagaev wrote:

> Hi , all
>
>I remember it was not recommended to add many columns (column
> qualifiers) in Hbase 0.18
> Does Hbase 0.19.0 still have this limitation?
>
> Thank you for your cooperation,
> M.
>


Re: Many columns in 0.19

2009-03-09 Thread Ryan Rawson
Don't forget, each column family is another file on disk, and file open.
Every column family is stored in it's own mapfile, and that increases the
load on HDFS.

This particular restriction won't ever really go away (unless we introduce
locality groups, even then, each locality group = N families = 1 file), but
in 0.20 it should be more feasable to have thousands of columns per family,
or more.

-ryan

On Mon, Mar 9, 2009 at 1:47 AM, Michael Dagaev wrote:

> Thank you, Ryan
>
> On Mon, Mar 9, 2009 at 10:28 AM, Ryan Rawson  wrote:
> > Sadly this is still a limit.
> >
> > 0.20 should make things much better.
> >
> > -ryan
> >
> > On Mon, Mar 9, 2009 at 12:23 AM, Michael Dagaev <
> michael.dag...@gmail.com>wrote:
> >
> >> Hi , all
> >>
> >>I remember it was not recommended to add many columns (column
> >> qualifiers) in Hbase 0.18
> >> Does Hbase 0.19.0 still have this limitation?
> >>
> >> Thank you for your cooperation,
> >> M.
> >>
> >
>


ganglia metrics for 'requests'

2009-03-11 Thread Ryan Rawson
Hi all,

So I eventually figured out that HBase is reporting the 'request' metric in
X second increments - 10 in my case from the config file.  Unfortunately
that doesn't really seem to be an ideal solution...

So I messed about, and I tried to replicate what was done in Hadoop
Datanode, but all I ended up with was a counter of how many requests - not a
rate.

So I can't figure this out - how do we get a request rate in ganglia?  Is
there a magic flag that needs to be set somewhere so ganglia can transform a
counter into a rate?

Or do we just need to do more work dividing our requests by how many seconds
between reporting intervals?

hel!
-ryan


Re: Import into empty table

2009-03-11 Thread Ryan Rawson
I have generally found in my imports that trying to aggressively split
doesnt seem to help.

I did write a file randomizer which seems to help.  It's a simple
map-reduce... I shoudl post it sometime.

-ryan

On Wed, Mar 11, 2009 at 1:14 AM, Mat Hofschen  wrote:

> Hi all,
> I am having trouble with importing a medium dataset into an empty new
> table.
> The import runs for about 60 minutes.
> There is a lot of failed/killed tasks in this scenario and sometime the
> import fails altogether.
>
> If I import a smaller subset into the empty table and then perform manual
> split of regions (via split button on webpage) and then import the larger
> dataset, the import runs for about 10 minutes.
>
> It seems to me that the performance bottleneck during the first import is
> the single region on the single cluster machine. This machine is heavily
> loaded. So my question is whether I can force hbase to split faster during
> heavy write operations and what tuning parameters may be affecting this
> scenario.
>
> Thanks for your help,
> Matthias
>
> p.s. here are the details
>
> Details:
> 33 cluster machines in testlab (3 year old servers with hyperthreading
> single core cpu) 1.5 gig of memory, debian 5 lenny 32bit
> hadoop 0.19.0, hbase 0.19.0
> -Xmx 500mb for java processes
> hadoop
> mapred.map.tasks=20
> mapred.reduce.tasks=15
> dfs.block.size=16777216
> mapred.tasktracker.map.tasks.maximum=4
> mapred.tasktracker.reduce.tasks.maximum=4
>
> hbase
> hbase.hregion.max.filesize=67108864
>
> hbase table
> 3 column families
>
> import file
> 5 Mill records with 18 columns with 6 columns per family
> filesize 1.1 gig csv-file
> import via provided java SampleUploader
>


Re: ganglia metrics for 'requests'

2009-03-11 Thread Ryan Rawson
I triesd time varying int and I just ended up with a monotonically
increasing request counter...

Mysterious!

On Mar 11, 2009 9:28 AM, "stack"  wrote:

There are different counter types up in the hadoop metrics package.  Maybe
we're using the wrong one (and they seem to have changed in hadoop 0.20.0).
St.Ack

On Wed, Mar 11, 2009 at 12:04 AM, Ryan Rawson  wrote: >
Hi all, > > So I event...


Re: Regarding IndexedTable.getIndexedScanner

2009-03-13 Thread Ryan Rawson
Right now the scanner only has 'next' not 'prev'.

So basically 'no'.  Try creating an index in the opposite order... you can
do Integer.MAX_VALUE - value to get a descending order.  Dont forget to 0
pad it too.

Good luck!
-ryan

On Thu, Mar 12, 2009 at 11:35 PM, Balamurugan Rajendran <
balamurug...@gmail.com> wrote:

> Dear
>
> Is it possible to get a Scanner which scans the record in descending order
> of a column value using IndexedTable.
>
> Thanks & Regards
> R.Bala
>


Re: Hbase on EC2 and issues with external access

2009-03-20 Thread Ryan Rawson
Yes if you patch src/ you need to rebuild.

this should be as simple as:
export JAVA_HOME=...
ant

and you are ready to go.

On Fri, Mar 20, 2009 at 12:46 PM, Yabo-Arber Xu wrote:

> I have upgraded it to 0.19.1, and apply the patch. A newbie question: do I
> have to re-build hbase to make it work?
>
> Best,
> Arber
>
> On Fri, Mar 20, 2009 at 12:20 PM, Jean-Daniel Cryans  >wrote:
>
> > Err you could try to apply the patch on 0.18.1 first but the best
> > would be to upgrade to 0.19.1
> >
> > J-D
> >
> > On Fri, Mar 20, 2009 at 3:10 PM, Yabo-Arber Xu  >
> > wrote:
> > > Hi J-D,
> > >
> > > Thank you very much. I shall try it. I am still using hbase-0.18.1. Do
> I
> > > have upgrade it to 0.19.x and then apply the patch?
> > >
> > > Best,
> > > Arber
> > >
> > > On Fri, Mar 20, 2009 at 7:51 AM, Jean-Daniel Cryans <
> jdcry...@apache.org
> > >wrote:
> > >
> > >> Arber,
> > >>
> > >> See my patch in https://issues.apache.org/jira/browse/HBASE-1279
> > >>
> > >> Now setting hbase.regionserver should work.
> > >>
> > >> J-D
> > >>
> > >>
> > >> On Fri, Mar 20, 2009 at 8:44 AM, Jean-Daniel Cryans <
> > jdcry...@apache.org>
> > >> wrote:
> > >> > I'm currently looking at the code and I think I see something that
> may
> > >> > be fixing that problem. We have a similar problem here so I'll check
> > >> > if it fixes it.
> > >> >
> > >> > J-D
> > >> >
> > >> > On Fri, Mar 20, 2009 at 5:51 AM, Yabo-Arber Xu <
> > arber.resea...@gmail.com>
> > >> wrote:
> > >> >> Hi J-D,
> > >> >>
> > >> >> I further found that the region server can actually be connected
> via
> > Web
> > >> UI
> > >> >> from an client outside Amazon network. That further verify the view
> > that
> > >> >> Master/Region server both are working, and it's just that Master
> > mapped
> > >> >> region servers' external IP into internal IP automatically, and the
> > >> clients
> > >> >> fail to communicate with region server via the mapped internal IP.
> > >> >>
> > >> >> Do you see any possible solution for this in the near future?
> > >> >>
> > >> >> Best,
> > >> >> Arber
> > >> >>
> > >> >> On Thu, Mar 19, 2009 at 4:48 PM, Yabo-Arber Xu <
> > >> arber.resea...@gmail.com>wrote:
> > >> >>
> > >> >>> Thx for your explanation. I suspect the reason is that when the
> > master
> > >> >>> initiate the contact with region server, region server was
> > recognized
> > >> >>> through its internal address ( as both of them are on Amazon
> > network),
> > >> even
> > >> >>> i explicitly put the external address for region server.
> > >> >>>
> > >> >>> So the consequence is is that the internal address shows in
> > namenode,
> > >> to
> > >> >>> which external clients can not connect.
> > >> >>>
> > >> >>>
> > >> >>> On Thu, Mar 19, 2009 at 4:33 PM, Jean-Daniel Cryans <
> > >> jdcry...@apache.org>wrote:
> > >> >>>
> > >>  Getting the "good" host name for a node is kind of a pain. For
> the
> > >>  moment, the implemented solution is that the Master tells the
> > region
> > >>  server to override it's known address with what the region server
> > was
> > >>  able to contact the master with. It was implemented like this so
> > that
> > >>  the region servers stop showing all up as "127.0.0.1" and hadoop
> > was
> > >>  doing it the same way.
> > >> 
> > >>  Now, I know that they changed it on their side, so I'd really
> > really
> > >>  like to know with which address the datanodes are showing up in
> the
> > >>  namenode. Internal or external? If it's the external one, I'll
> dive
> > in
> > >>  their code.
> > >> 
> > >>  Thx,
> > >> 
> > >>  J-D
> > >> 
> > >>  On Thu, Mar 19, 2009 at 7:26 PM, Yabo-Arber Xu <
> > >> arber.resea...@gmail.com>
> > >>  wrote:
> > >>  > Hi J-D,
> > >>  >
> > >>  > Thanks for your reply, and sorry for my late response as I was
> > >> engaged
> > >>  in
> > >>  > sth else in the past two days.
> > >>  >
> > >>  > Yes, I've unlocked the port, and i am actually able to access
> > from
> > >> the
> > >>  web
> > >>  > UI with a client not running on EC2 to HBase at
> > example.com:60010.
> > >> It
> > >>  shows
> > >>  > all User Tables, but the Region Servers Address is the EC2
> > internal
> > >>  address:
> > >>  > domU-12-31-39-00-65-E5.compute-1.internal:60020.
> > >>  >
> > >>  > I guess the client fails because it can not connect region
> > server,
> > >> which
> > >>  > serves only for an internal IP. However, in hbase-site.xml, I
> did
> > >>  configure
> > >>  > with region server explicitly in its external IP.
> > >>  >
> > >>  >  ^M
> > >>  >hbase.regionserver
> > >>  >ec2-67-202-57-127.compute-1.amazonaws.com:60020
> > 
> > >>  >The host and port a HBase region server runs
> > at.^M
> > >>  >
> > >>  >  
> > >>  >
> > >>  > What could I do wrong?
> > >>  >
> > >>  > Thanks again,
> > >>  > Arber
> > >>  >
> > >>  >

Re: HBase new user

2009-03-22 Thread Ryan Rawson
Hey,

Here are some answers:
1. The directory setting is:
hbase.rootdir
file:///tmp/hbase-${user.name}/hbase
The directory shared by region servers.
Should be fully-qualified to include the filesystem to use.
E.g: hdfs://NAMENODE_SERVER:PORT/HBASE_ROOTDIR


adjust that in your hbase-site.xml as you wish.  I have mine set to:
hdfs://master:9000/hbase

It is conceivable to run HBase on a shared-directory system like NFS, but I
haven't heard of anyone doing that.

2.  There is a thrift API, which you can hook up any thrift-client to. This
includes perl.

3.  The docs online, try irc, there are a number of things there.

4.  What kind of GUI would you like?  You could write your own web one?
There is the bin/hbase shell which gives admin access to the system - do you
need more than that?




On Sun, Mar 22, 2009 at 2:29 PM, Jae Joo  wrote:

> Hi,
>
> I am new for HBase and have a couple of questions.
> 1. I have installed HBase and load a couple of millions of data. The file
> system for HBase is in /tmp diectory.
>   How can I change the directory for data files?
> 2. Is there any Perl based API available?
> 3. Is there any tutorial available?
> 4. Does anyone know that for GUI based API is available?
>
>
> Thanks,
>
> Jae Joo
>


Re: RDF store over HDFS/HBase

2009-03-23 Thread Ryan Rawson
I would expect HBase would scale well - the semantics of the data being
stored shouldn't matter, just the size.

I think there are a number of production HBase installations that have
billions of rows.

On Mon, Mar 23, 2009 at 4:10 PM, Ding, Hui  wrote:

> I remember there was a project proposal back in late last year.  They've
> set up an official  webpage.Not sure if they are still alive/making any
> progress.
> You  can search in the email archive.
>
> -Original Message-
> From: Amandeep Khurana [mailto:ama...@gmail.com]
> Sent: Monday, March 23, 2009 4:07 PM
> To: hbase-user@hadoop.apache.org; core-u...@hadoop.apache.org;
> core-...@hadoop.apache.org
> Subject: RDF store over HDFS/HBase
>
> Has anyone explored using HDFS/HBase as the underlying storage for an
> RDF
> store? Most solutions (all are single node) that I have found till now
> scale
> up only to a couple of billion rows in the Triple store. Wondering how
> Hadoop could be leveraged here...
>
> Amandeep
>
>
> Amandeep Khurana
> Computer Science Graduate Student
> University of California, Santa Cruz
>


Re: HDFS unbalance issue. (HBase over HDFS)

2009-03-25 Thread Ryan Rawson
Try
hadoop/bin/start-balancer.sh

HDFS doesnt auto-balance.  Balancing in HDFS requires moving data around,
whereas balancing in HBase just means opening a file on a different machine.

On Wed, Mar 25, 2009 at 2:12 AM, schubert zhang  wrote:

> Hi all,
> I am using hbase-0.19.1 and hadoop-0.19.
> My cluster have 5+1 nodes, and there are about 512 regions in HBase (256MB
> per region).
>
> But I found the blocks in HDFS is very unbalanced. Following is the status
> from HDFS web GUI.
>
> (Node: I don't know if this mailing list can display html!)
>
> HDFS blocks:
> node1   509036 blocks
> node2   157937 blocks
> node3   15783   blocks
> node4   15117   blocks
> node5   20158   blocks
>
> But my HBase regions are very balanced.
> node1   88   regions
> node2   108 regions
> node3   111 regions
> node4   102 regions
> node5   105 regions
>
>
>
> NodeLast
> ContactAdmin StateConfigured
> Capacity (GB)Used
> (GB)Non DFS
> Used (GB)Remaining
> (GB)Used
> (%)Used
> (%)Remaining
> (%)Blocksnd1-rack0-cloud<
> http://nd1-rack0-cloud:50075/browseDirectory.jsp?namenodeInfoPort=50070&dir=%2F
> >
> 0In Service822.8578.6743.28200.8670.3324.41509036nd2-rack0-cloud<
> http://nd2-rack0-cloud:50075/browseDirectory.jsp?namenodeInfoPort=50070&dir=%2F
> >
> 0In Service822.8190.0242.96589.8223.0971.68157937nd3-rack0-cloud<
> http://nd3-rack0-cloud:50075/browseDirectory.jsp?namenodeInfoPort=50070&dir=%2F
> >
> 0In Service822.851.9542.61728.246.3188.5115783nd4-rack0-cloud<
> http://nd4-rack0-cloud:50075/browseDirectory.jsp?namenodeInfoPort=50070&dir=%2F
> >
> 6In Service822.846.1942.84733.775.6189.1815117nd5-rack0-cloud<
> http://nd5-rack0-cloud:50075/browseDirectory.jsp?namenodeInfoPort=50070&dir=%2F
> >
> 1In Service1215.6152.3762.911100.324.3190.5220158
>
>
> But my HBase regions are very balanced.
>
> AddressStart CodeLoadnd1-rack0-cloud:60020 
> 1237967027050requests=383, regions=88, usedHeap=978, maxHeap=1991
> nd2-rack0-cloud:60020  >1237788871362requests=422,
> regions=108, usedHeap=1433,
> maxHeap=1991nd3-rack0-cloud:60020
> 123771667requests=962, regions=111, usedHeap=1534, maxHeap=1991
> nd4-rack0-cloud:60020  >1237788859541requests=369,
> regions=102, usedHeap=1059,
> maxHeap=1991nd5-rack0-cloud:60020
> 1237788899331requests=384, regions=105, usedHeap=1535,
> maxHeap=1983Total:servers:
> 5 requests=2520, regions=514
>


Re: Question on write optimization

2009-03-25 Thread Ryan Rawson
Hey,

At the lowest level, hbase is append only, so multiple values just end up
taking extra space (not a big deal) and get compacted out eventually.

It would seem to me that the cost of the reads would in many cases higher
than the cost of extra writes that would have to get compacted out later -
there must be a tipping point, but checking every read before write is
pretty brutal.  You are doubling the number of operations you must do.

I find that commit buffering is pretty efficient - I can import 880m rows in
about 90 minutes on a 19 machine cluster.

you can access this in Java by:
table.setAutoCommit(false);
table.setAutoCommitBuffer(12 * 1024 * 1024); // i think i got this method
name wrong.

-ryan

On Wed, Mar 25, 2009 at 2:18 AM, Michael Dagaev wrote:

> Hi, all
>
>Currently, we write all incoming entities using batch update.
>
> Recently we realized that many incoming entities already exist.
> So, we can check for each incoming entity if it already exists
> and write only "new" entities.
>
> In other words, we can perform more reads and less writes now.
> Does it make sense?
>
> Thank you for your cooperation,
> M.
>


Re: no memory tables

2009-03-26 Thread Ryan Rawson
Hey,

Interesting ideas - there are some features in 0.20 that might obviate the
need for some of the suggestions below...

One major problem with hbase 0.19 is the indexing scheme - an index entry is
created every 128 entries.  With large data sets with small key/values, this
is a major problem.

But in hbase 0.20, the index is now based on blocks.  On my own test:
- 1 hfile that is 161 MB on disk
- contains 11m key/values
- represents about 5.5 million rows
- 3.7x compression
- default block size (pre-compression) of 64kBytes
- in-memory block index size: 770kBytes.

One problem with 0.19 is the size of in-memory indexes... With hfile in 0.20
we will have many less problems.


On Thu, Mar 26, 2009 at 11:20 PM, Billy Pearson
wrote:

> I was wondering if anyone else out there would like to use hbase to support
> storing data that does not need random access just insert/delete/scan
> If we could support a table like this that would require little to no
> memory but still allow sorted scanable updateable data to be
> stored in hbase with out the need to have index of keys in memory.
> We should still have memory usage with inserts stored in memcache but no
> key index in memory.
>
> This would allow large datasets that do not need random access to be stored
> and still give access to new/live
> data with scans with out having to merge/sort the data on disk manually
> before seeing updates.
>
> I have a large amount of data coming in that needs expired over time. I
> store in hadoop and run MR jobs over it to produce accessible index of the
> data via hbase.
> The ideal here is if I could import that data in to hbase then I can access
> subsets of the data with out having to read all the data to find what I am
> looking for.
> with this hbase could merge/sort/expire/split the data as needed and still
> give access to newly inserted data.
>
> This might take some memory on the master node but I would not thank there
> would be a limit on the size of the data except the hadoop storage size.
> Anyone else thank they could use something like this also?
>
> Billy Pearson
>
>
>
>


Re: no memory tables

2009-03-27 Thread Ryan Rawson
Trunk is workable right now - but ymmv and there is no, and I repeat, NO
guarantee the file format won't change at a moment's notice.  I mean it -
you could svn up and your installation will be trashed, with no way to
recover it except rm -rf /hbase.

In fact, let me stress again - there is an outstanding patch that will
infact change the basic storage format (key format in hfile to be
specific).

BUT

If you are willing to toss test data in a throw-away instance, give it a
spin.  Watch as your 25 node cluster sustains 200-300k ops/sec for hours on
end (turn compression on to 'gz').  Be amazed as scanners return 0 rows in
0ms from the client's point of view.  And so on.

Last piece, on my data from earlier, I forgot to mention that the rowid size
is 16 bytes, and the data varys, but is probably no more than 20-30 bytes or
so.  Column names are 'default:0', so total size per row is like
16+20-30+8+10 = 54-64 bytes.



On Fri, Mar 27, 2009 at 1:14 AM, Andrew Purtell  wrote:

>
> I'm really looking forward to taking HFile for a spin. Thanks so
> much for your contributions, Ryan.
>
>  - Andy
>
> > From: Ryan Rawson 
> > Subject: Re: no memory tables
> > To: hbase-user@hadoop.apache.org
> > Date: Thursday, March 26, 2009, 11:31 PM
> > Hey,
> >
> > Interesting ideas - there are some features in 0.20 that
> > might obviate the need for some of the suggestions below...
> >
> > One major problem with hbase 0.19 is the indexing scheme -
> > an index entry is created every 128 entries.  With large
> > data sets with small key/values, this is a major problem.
> >
> > But in hbase 0.20, the index is now based on blocks.  On my
> > own test:
> > - 1 hfile that is 161 MB on disk
> > - contains 11m key/values
> > - represents about 5.5 million rows
> > - 3.7x compression
> > - default block size (pre-compression) of 64kBytes
> > - in-memory block index size: 770kBytes.
> >
> > One problem with 0.19 is the size of in-memory indexes...
> > With hfile in 0.20 we will have many less problems.
>
>
>
>
>


Re: Iterating over all cells in HBase

2009-03-27 Thread Ryan Rawson
The RowResult class contains what you are looking for...

you want 'entrySet()' which contains byte[] -> Cell, where byte[] = column
name, Cell = value & timestamp of the data.

Good luck!
-ryan

On Fri, Mar 27, 2009 at 1:28 PM, Vaibhav Puranik  wrote:

> Hi all,
>
> One more newbie question.
>
> Is there any way to iterate over all the cells (values of all column
> families:all columns) of a given table without knowing what columns there
> are?
> (I am still struggling with whether a column is data or metadata)
>
> I looked into apis for HTable, HTableDescriptor, HColumnDescriptor but
> couldn't find anything to get columns.
>
> tables.each {tableDescriptor ->
>families = tableDescriptor.getFamilies()
>families.each {family ->
>print family.getNameAsString()
>}
> }
>
> Can anyone suggest a method in HColumnDescriptor that can give me all the
> column names?
>
> Is it unusual to iterate over all column, all rows in a given HBase table?
> We were thinking of doing that to populate memcached.
>
> Regards,
> Vaibhav
>


Re: Typical hardware configurations

2009-03-27 Thread Ryan Rawson
Even though hbase runs on 'commodity' hardware, it's important to remember
that to achieve scale you need to do a bit better than 1 cpu 1 gb ram type
things.

I tend to think in per-core specs, that way you don't have to worry about 2
core vs 4 core vs 8 core - you buy whatever is most economical at the time.

I'd match 1 core with 2-4gb ram.  You'll want to dedicate 4 gb of ram to
hbase, it'll make life easier.

You should also be getting 64-bit systems and running a 64 bit distro on it
and a jvm that has -d64 available.

A word about master... For hbase, the master is (a) important and (b) very
lightweight.  Meaning the master doesn't use much ram.For hadoop, things
are different, because the HDFS master is relatively light weight, but needs
lots of ram (every file takes up memory space).  On my cluster, the master
is the same node-type as the rest.

I've heard recommendations to buy better hardware for your master - if you
lose a disk, your whole cluster goes down.  I can't say i disagree with that
sentiment.

Good luck!

On Fri, Mar 27, 2009 at 10:43 PM, Yabo-Arber Xu wrote:

> Hi Amandeep,
>
> I just did the same investigation not long ago, and I was recommended to
> get
> Amazon EC2 X-Large
> equivalent<
> http://www.google.com/url?q=http%3A%2F%2Faws.amazon.com%2Fec2%2F%23pricing&sa=D&sntz=1&usg=AFrqEzc1z8IB5p0hIR7SGe-mRVRZXW7Lvg
> >nodes:
> , 8
> EC2 Compute Units (4 virtual cores with 2 EC2 Compute Units each), 15 GB
> memory, 1690 GB of instance storage, 64-bit platform. One EC2 Compute Unit
> (ECU) is  equivalent to CPU capacity of a 1.0-1.2 GHz 2007 Opteron or 2007
> Xeon processor.
>
> For more details, you may want to refer to Daniel Leffel's experience on
> setting up HBase<
> http://www.google.com/url?q=http%3A%2F%2Fmail-archives.apache.org%2Fmod_mbox%2Fhadoop-hbase-user%2F200805.mbox%2F%253C25e5a0c00805072129w3b54599r286940f134c6f235%40mail.gmail.com%253E&sa=D&sntz=1&usg=AFrqEzcmU5_eMlrfoBJwCTxOg9I8NeJ2JQ
> >
>
> Hope it helps.
>
> Best,
> Arber
>
> On Fri, Mar 27, 2009 at 10:07 PM, Amandeep Khurana 
> wrote:
>
> > What are the typical hardware config for a node that people are using for
> > Hadoop and HBase? I am setting up a new 10 node cluster which will have
> > HBase running as well that will be feeding my front end directly.
> > Currently,
> > I had a 3 node cluster with 2 GB of RAM on the slaves and 4 GB of RAM on
> > the
> > master. This didnt work very well due to the RAM being a little low.
> >
> > I got some config details from the powered by page on the Hadoop wiki,
> but
> > nothing like that for Hbase.
> >
> >
> > Amandeep Khurana
> > Computer Science Graduate Student
> > University of California, Santa Cruz
> >
>


Re: How is Hbase column oriented ?

2009-04-01 Thread Ryan Rawson
Hey,

The hbase wiki has some good docs, you can start with:
http://wiki.apache.org/hadoop/Hbase/HbaseArchitecture

Also don't forget to read the bigtable paper:
http://labs.google.com/papers/bigtable.html

Good luck!

2009/4/1 yamada hiroyuki 

> Hello,
>
> I'm currently studying Hbase/BigTable .
> I kind of understand their conceptual data structure, but
> I don't get how it's physically implemented, especially
> how it's column oriented.
> I referenced BigTable paper and Hbase wiki, but
> none of them describe it.
> (it's just saying column oriented, I think.)
>
> Does anyone teach me how it's implemented ?
> (or you could guide me any pages describing it .)
>
> Thanks,
> Hiro
>


Re: Bulk import - does sort order of input data affect success rate?

2009-04-02 Thread Ryan Rawson
The last thing - success should not be a function of sort order.

However, speed will be related.

One thing I found I had to do was:
private void doCommit(HTable t, BatchUpdate update) throws IOException {
  boolean commited = false;
  while (!commited) {
try {
  t.commit(update);
  commited = true;
} catch (RetriesExhaustedException e) {
  // DAMN, ignore
}
  }
}

good luck!
-ryan

On Thu, Apr 2, 2009 at 1:28 PM, Stuart White wrote:

> I, like many others, am having difficulty getting a mapred job that
> bulk imports data into an HBase table to run successfully to
> completion.
>
> At this time, rather than get into specifics of my configuration, the
> exceptions I'm receiving, etc..., I wanted to ask a general question:
>
> Should I expect my bulk import to be more likely to succeed if my data
> is sorted by its key?
> Or should I expect my bulk import to be more likely to succeed if my
> data is randomized?
> Or should I expect the ordering of my input data to have no effect on
> my ability to successfully bulk import records?
>
> Thanks.
>


Re: Monitoring xceivers

2009-04-02 Thread Ryan Rawson
That should work - the threads are indeed native, so they will show up in
the OS internals.

On Thu, Apr 2, 2009 at 6:29 AM, Michael Dagaev wrote:

> Hi, all
>
>   I heard from somebody on the list that the xceivers number
>  is actually the number of worker threads of the data node process.
>
> Assuming Java uses native threads we can monitor the number of xceivers
> just by ls /proc//task | wc -l and if it is getting to the
> limit,
> add data  nodes.
>
> Does it make sense?
> M.
>


Re: Bulk import - does sort order of input data affect success rate?

2009-04-02 Thread Ryan Rawson
hey,

sorted = slower, randomized = faster.

this is because if it is sorted in natural key order, you tend to hotspot in
1 or 2 regions.

I don't use table output format, I use direct commits from the map, no
reduce. That seems to be the most performance solution.

have fun!


On Thu, Apr 2, 2009 at 1:36 PM, Stuart White wrote:

> On Thu, Apr 2, 2009 at 3:30 PM, Ryan Rawson  wrote:
> > The last thing - success should not be a function of sort order.
> >
> > However, speed will be related.
>
> How?  Sorted = faster, or Sorted = slower?
>
> >
> > One thing I found I had to do was:
> >private void doCommit(HTable t, BatchUpdate update) throws IOException
> {
> >  boolean commited = false;
> >  while (!commited) {
> >try {
> >  t.commit(update);
> >  commited = true;
> >} catch (RetriesExhaustedException e) {
> >  // DAMN, ignore
> >}
> >  }
> >}
> >
>
> I'm running a mapred job, using TableOutputFormat to write the results
> to HBase.  For the code you've provided, was that for a custom output
> format?  Or a standalone (non-mapred) application?  I see the point
> you're making, I just don't understand where I'd put that code.
> Thanks!
>


Re: financial time series database

2009-04-03 Thread Ryan Rawson
Another reason to perhaps avoid tons of versions is there is no query
mechanism, nor will there ever be.  The mechanism is limited to asking for
either the last N versions, or all of them.  If you are querying a date
range, this is obviously a problem.

-ryan

On Fri, Apr 3, 2009 at 7:25 AM, stack  wrote:

> On Thu, Apr 2, 2009 at 9:53 PM, Wesley Chow  wrote:
>
> >
> > Are there technical limitations to the number of different timestamps per
> > cell? If it's the case that you're doing to be dealing with tens of
> > thousands to millions of entries all at one cell, perhaps you should
> check
> > that to make sure it's a reasonable use case. The examples in the HBase
> docs
> > number the timestamps in single digits, and I don't recall any mention of
> > very large numbers.
>
>
>
> Agreed.  I'd  imagine that tens of thousands of versions currently would
> suffer in same manner as tens of thousands of columns -- hbase running
> increasingly slower as count went up, at least until we address
> HBASE-867 "If
> millions of columns in a column family, hbase scanner won't come
> up
> "
>
>
> St.Ack
>


Re: Still need help with data upload into HBase

2009-04-06 Thread Ryan Rawson
I ran into a problem on ubuntu where /etc/security/limits.conf wasnt being
honored due to a missing line in /etc/pam.d/common-session:
"session requiredpam_limits.so"

this prevented the ulimits from being run.

can you sudo to the hadoop/hbase user and verify with ulimit -a ?



On Mon, Apr 6, 2009 at 5:07 PM, Taylor, Ronald C wrote:

>  Hello Ryan and the list,
>
> Well, I am still stuck. In addition to making the changes recommended by
> Ryan to my hadoop-site.xml file (see below), I also added a line for HBase
> to /etc/security/limits.conf and had the fs.file-max hugely increased, to
> hopefully handle any file handle limit problem. Still no luck with my upload
> program. It fails about where it did before, around the loading of  the
> 160,000th row into the one table that I create in Hbase. Didn't  the "too
> many file open" msg, but did get "handleConnectionFailure"  in the same
> place in the upload.
>
> I then tried a complete reinstall of Hbase and Hadoop, upgrading from
> 0.19.0 to 0.19.1. Used the same config parameters as before, and reran the
> program. It fails again, at about the same number of rows uploaded - and I'm
> back to getting "too many files open" as what I think is the principal error
> msg.
>
> So - does anybody have any suggestions? I am running a "pseudo-distributed"
> installation of Hadoop on one Red Hat Linux machine with about ~3Gb of RAM.
> Are there any known problems with bulk uploads when running
> "pseudo-distributed" on on a single box, rather than a true cluster? Is
> there anything else I can try?
>  Ron
>
>
> ___
> Ronald Taylor, Ph.D.
> Computational Biology & Bioinformatics Group
> Pacific Northwest National Laboratory
> 902 Battelle Boulevard
> P.O. Box 999, MSIN K7-90
> Richland, WA  99352 USA
> Office:  509-372-6568
> Email: ronald.tay...@pnl.gov
> www.pnl.gov
>
>
>  --
> *From:* Ryan Rawson [mailto:ryano...@gmail.com]
> *Sent:* Friday, April 03, 2009 5:56 PM
> *To:* Taylor, Ronald C
> *Subject:* Re: FW: Still need help with data upload into HBase
>
> Welcome to hbase :-)
>
> This is pretty much how it goes for nearly every new user.
>
> We might want to review our docs...
>
> On Fri, Apr 3, 2009 at 5:54 PM, Taylor, Ronald C wrote:
>
>>
>>
>> Thanks. I'll make those settings, too, in addition to bumping up the
>> file handle limit, and give it another go.
>> Ron
>>
>> -Original Message-
>> From: Ryan Rawson [mailto:ryano...@gmail.com]
>> Sent: Friday, April 03, 2009 5:48 PM
>> To: hbase-user@hadoop.apache.org
>>  Subject: Re: Still need help with data upload into HBase
>>
>> Hey,
>>
>> File handle - yes... there was a FAQ and/or getting started which talks
>> about upping lots of limits.
>>
>> I have these set in my hadoop-site.xml (that is read by datanode):
>> 
>> dfs.datanode.max.xcievers
>> 2047
>> 
>>
>> 
>> dfs.datanode.handler.count
>> 10
>> 
>>
>> I should probably set the datanode.handler.count higher.
>>
>> Don't forget to toss a reasonable amount of ram at hdfs... not sure what
>> that is exactly, but -Xmx1000m wouldn't hurt.
>>
>> On Fri, Apr 3, 2009 at 5:44 PM, Taylor, Ronald C
>> wrote:
>>
>> >
>> > Hi Ryan,
>> >
>> > Thanks for the info. Re checking the Hadoop datanode log file: I just
>> > did so, and found a "too many open files" error. Checking the Hbase
>> FAQ,
>> > I see that I should drastically bump up the file handle limit. So I
>> will
>> > give that a try.
>> >
>> > Question: what does the xciver variable do? My hadoop-site.xml file
>> does
>> > not contain any entry for such a var. (Nothing reported in the datalog
>> > file either with the word "xciver".)
>> >
>> > Re using the local file system: well, as soon as I load a nice data
>> set
>> > loaded in, I'm starting a demo project manipulating it for our Env
>> > Molecular Sciences Lab (EMSL), a DOE Nat User Facility. And I'm
>> supposed
>> > to be doing the manipulating using MapReduce programs, to show the
>> > usefulness of such an approach. So I need Hadoop and the HDFS. And so
>> I
>> > would prefer to keep using Hbase on top of Hadoop, rather than the
>> local
>> > Linux file system. Hopefully the "small HDFS clusters" issues you
>> > mention are survivable. Eventually, some of this

Re: help with map-reduce

2009-04-07 Thread Ryan Rawson
there is a server-side mechanism to filter rows, it's found in the
org.apache.hadoop.hbase.filter package.  im not sure how this interops with
the TableInputFormat exactly.

setting a filter to reduce the # of rows returned is pretty much exactly
what you want.

On Tue, Apr 7, 2009 at 2:26 AM, Rakhi Khatwani wrote:

> Hi,
> i have a map reduce program with which i read from a hbase table.
> In my map program i check if the column value of a is xxx, if yes then
> continue with processing else skip it.
> however if my table is really big, most of my time in the map gets wasted
> for processing unwanted rows.
> is there any way through which we could send a subset of rows (based on the
> value of a particular column family) to the map???
>
> i have also gone through TableInputFormatBase but am not able to figure out
> how do we set the input format if we are using TableMapReduceUtil class to
> initialize table map jobs. or is there any other way i could use it.
>
> Thanks in Advance,
> Raakhi.
>


Re: Region Servers going down frequently

2009-04-08 Thread Ryan Rawson
Just FYI, 0.20 handles small cell values substantially better than 0.19.1.

-ryan

On Wed, Apr 8, 2009 at 12:35 AM, Amandeep Khurana  wrote:

> Hadoop and hbase are intelligent enough to balance the load. Its not very
> frequent that you need to balance the load manually. Your cluster isnt
> performing because of the low memory and the low limits on top of it. I
> dont
> think the load is a problem at all.
>
> Hadoop and Hbase are not designed for small data sizes and therefore dont
> have the best performance when you have small files or small tables. The
> most difficult part of hbase is starting up and increasing the table to a
> certain threshold level. You'll encounter troubles in that phase (which you
> already are). After that, its a breeze...
>
>
> Amandeep Khurana
> Computer Science Graduate Student
> University of California, Santa Cruz
>
>
> On Wed, Apr 8, 2009 at 12:29 AM, Rakhi Khatwani  >wrote:
>
> > Thanks, Amandeep
> >
> > One more question, i have mailed it earlier and i have attached the
> > snapshot
> > along with that email.
> > I have noticed it that all my requests are handled by one region
> server...
> > Is there any way to balance the load?
> > and will balancing the load improve the performance?
> >
> > PS: I have tried using hadoop load balancing but after some time some of
> my
> > region servers shut down... i have even gone through the archives and
> > someone did report an unstable cluster due to load balancing. so i really
> > dont know if i should turn load balancing on?
> >
> > Thanks,
> > Raakhi
> >
> > On Wed, Apr 8, 2009 at 12:51 PM, Amandeep Khurana 
> > wrote:
> >
> > > I'm not sure if I can answer that correctly or not. But my guess is no
> it
> > > wont hamper the performance.
> > >
> > >
> > > Amandeep Khurana
> > > Computer Science Graduate Student
> > > University of California, Santa Cruz
> > >
> > >
> > > On Wed, Apr 8, 2009 at 12:13 AM, Rakhi Khatwani <
> > rakhi.khatw...@gmail.com
> > > >wrote:
> > >
> > > > Hi Amandeep,
> > > >
> > > > But in That case, if I let hbase split it automatically, my table
> with
> > > > 17000
> > > > rows will have only one region. thus my analysis will have only one
> > map.
> > > > won't the analysis process be slower in that case??
> > > >
> > > > Thanks,
> > > > Raakhi
> > > >
> > > > On Wed, Apr 8, 2009 at 12:35 PM, Amandeep Khurana 
> > > > wrote:
> > > >
> > > > > You cant compensate the RAM with processing power. Hbase keeps a
> lot
> > of
> > > > > open
> > > > > file handles in hdfs which needs memory so you need the RAM.
> > > > >
> > > > > Secondly, 17000 rows isnt much to cause a region split. I dont know
> > > exact
> > > > > numbers but I had a table with 6 million rows and only 3 regions.
> So,
> > > > thats
> > > > > not a big deal.
> > > > >
> > > > > Thirdly, try with upping the xceivers and ulimit and see if it
> works
> > > with
> > > > > the existing RAM... Thats the only way out.
> > > > >
> > > > >
> > > > > Amandeep Khurana
> > > > > Computer Science Graduate Student
> > > > > University of California, Santa Cruz
> > > > >
> > > > >
> > > > > On Wed, Apr 8, 2009 at 12:02 AM, Rakhi Khatwani <
> > > > rakhi.khatw...@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > Hi Amandeep,
> > > > > >
> > > > > > Following is my ec2 cluster configuration:
> > > > > > High-CPU Medium Instance 1.7 GB of memory, 5 EC2 Compute Units (2
> > > > virtual
> > > > > > cores with 2.5 EC2 Compute Units each), 350 GB of instance
> storage,
> > > > > 32-bit
> > > > > > platform
> > > > > >
> > > > > > so I don't think I have much option when it comes to the GB part.
> > > > > iHowever,
> > > > > > is there any way i can make use of 5ec2 compute units to increase
> > my
> > > > > > performance?
> > > > > >
> > > > > > Regarding the table splits, I dont see hbase doing the table
> spilts
> > > > > > automatically.
> > > > > > After loading about 17000 rows in table1, I can still see it as
> one
> > > > > region
> > > > > > (after checking it on web UI). thats why i had to manually split
> > it.
> > > or
> > > > > is
> > > > > > there any configuration/settings I have to do to ensure that the
> > > tables
> > > > > are
> > > > > > split automatically?
> > > > > >
> > > > > > I will increase the dataXceivers and ulimit to 32k
> > > > > >
> > > > > > Thanks a ton
> > > > > > Rakhi.
> > > > > >
> > > > > >
> > > > > >
> > > > > > >
> > > > > > > > Hi Amandeep,
> > > > > > > >  I have 1GB Memory on each node on ec2
> > cluster(C1
> > > > > > Medium)
> > > > > > > .
> > > > > > > > i am using hadoop-0.19.0 and hbase-0.19.0
> > > > > > > > well we were starting with 10,000 rows, but later it will go
> up
> > > to
> > > > > > > 100,000
> > > > > > > > rows.
> > > > > > >
> > > > > > >
> > > > > > > 1GB is too low. You need around 4GB to get a stable system.
> > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > my map task basically reads an hbase table 'Table1', performs
> > > > > analysis
> > > > > > on
> > > > > > >

Re: setting xciever number limit

2009-04-09 Thread Ryan Rawson
sic is latin for 'so, thus'.  meaning, that the word looks odd or wrong, but
it is quote exactly as it is in the original.

Of course, an xciever is something that transmits and receives, but
apparently i before e, except after c didnt apply this time.

On Thu, Apr 9, 2009 at 1:22 PM, Lars George  wrote:

> Hi Ron,
>
> I am adding it to indicate the misspelling. Usually these keywords are of
> course in the hadoop-defaults.xml, but that seems not to be the case with
> this rather new one. Maybe the 0.19.1 and newer is all good, my 0.19.0 does
> not have it though.
>
> And to clarify the spelling, here the code from Hadoop's
> DataXceiverServer.java:
>
>   this.maxXceiverCount = conf.getInt("dfs.datanode.max.xcievers",
>   MAX_XCEIVER_COUNT);
>
> HTH,
>
> Lars
>
>
> Taylor, Ronald C wrote:
>
>>  Hi Lars,
>>
>> I looked up the Troubleshooting entry, as you suggested. It says
>> "dfs.datanode.max.xcievers (sic)". Um ... isn't the "(sic)" supposed to
>> indicate that the *wrong* spelling is being used in the previous phrase?
>> Or is the "(sic)" being used here to signify that yes, we know this is
>> an unexpected spelling, but use it anyway?
>> Ron
>>
>> -Original Message-
>> From: Lars George [mailto:l...@worldlingo.com] Sent: Thursday, April 09,
>> 2009 1:09 AM
>> To: hbase-user@hadoop.apache.org
>> Cc: Taylor, Ronald C
>> Subject: Re: Still need help with data upload into HBase
>>
>> Hi Ron,
>>
>> The syntax is like this (sic):
>>
>>
>>dfs.datanode.max.xcievers
>>4096
>>
>>
>> and it is documented on the HBase wiki here:
>> http://wiki.apache.org/hadoop/Hbase/Troubleshooting
>>
>> Regards,
>> Lars
>>
>>
>> Taylor, Ronald C wrote:
>>
>>
>>>  Hi Ryan,
>>>
>>> Thanks for the suggestion on checking whether the number of file handles
>>> allowed actually gets increased after I make the change the
>>> /etc/security/limits.conf.
>>>
>>> Turns out it was not. I had to check with one of our sysadmins so that
>>>
>>>
>>
>>
>>
>>> the new 32K number of handles setting actually gets used on my Red Hat
>>>
>>>
>>
>>
>>
>>> box.
>>>
>>> With that, and with one other change which I'll get to in a moment, I
>>> finally was able to read in all the rows that I wanted, instead of the
>>>
>>>
>>
>>
>>
>>> program breaking before finishing. Checked the table by scanning it -
>>> looks OK. So - it looks like things are working as they should.
>>>
>>> Thank you very much for the help.
>>>
>>> Now as to the other parameter that needed changing: I found that the
>>> xceivers (xcievers?) limit was not being bumped up - I was crashing on
>>>
>>>
>>
>>
>>
>>> that. I went to add what Ryan suggested in hadoop-site.xml, i.e.,
>>>
>>> 
>>> dfs.datanode.max.xcievers
>>> 2047
>>> 
>>>
>>> and discovered that I did not know whether to use
>>> "dfs.datanode.max.xcievers" or "dfs.datanode.max.xceivers", where the "i"
>>> and "e" switch. I was getting error msgs in the log files with
>>>
>>>  "xceiverCount 257 exceeds the limit of concurrent xcievers 256"
>>>
>>>  with BOTH spelling variants employed within the same error msg. Very
>>> confusing. So I added property entries for both spellings in the
>>> hadoop-site.xml file. Figured one of them would take effect. That appears to
>>> work fine. But I would like get the correct spelling. Did a
>>>
>>>
>>
>>
>>
>>> Google search and the spelling keeps popping up both ways, so I remain
>>>
>>>
>>
>>
>>
>>> confused.
>>>
>>> I think the Hbase getting started documentation could use some
>>> enhancement on file handle settings, xceiver (xciever?) settings, and
>>> datanode handler count settings.
>>>  Ron
>>>
>>> ___
>>> Ronald Taylor, Ph.D.
>>> Computational Biology & Bioinformatics Group Pacific Northwest National
>>> Laboratory
>>> 902 Battelle Boulevard
>>> P.O. Box 999, MSIN K7-90
>>> Richland, WA  99352 USA
>>> Office:  509-372-6568
>>> Email: ronald.ta

Re: How to check the distributed degree of table?

2009-04-09 Thread Ryan Rawson
Hey,

In HBase, each table is split into regions.  Each region is a contiguous set
of keys.  Once any specific region has a file that exceeds 256 MB, it is
split in half to 2 regions. HBase master generally spreads these regions
across all the regionservers.  I have not had problems with hot
regionservers with too many regions.

When a table starts out, it has 1 region.  If you dont hit the 256MB limit,
then you won't split, and thus wont have more than 1 region to distribute
across tables.

One way to get a handle on how many regions a table might have is querying
.META. - if you search for the key 'table_name,,' and for the column
'info:regioninfo' you can get a maximal bound (but not exact count) of how
many regions your table has.  To get the exact count, you have to parse the
info:regioninfo and detect the split parent regions that don't really
'exist' but are kept around for garbage collection later.

Just FYI, with 400m rows I get about 200-300 regions given a value size of
about 30 bytes.  10,000 rows may not have been enough to trigger a split if
the values are small.

On Thu, Apr 9, 2009 at 4:48 PM, Edward J. Yoon wrote:

> Can it be provided to clients as a API?
>
> I need to consecutively run two MR jobs and I would like to add some
> precondition checks in between since I noticed that second job always
> run on a single node.
>
> On Thu, Apr 9, 2009 at 8:07 PM, Jean-Daniel Cryans 
> wrote:
> > I usually look in the web UI, clicking on the name of the table in the
> > master page, to see if it's well distributed.
> >
> > Or are you thinking about a shell tool that tells you the level of
> > distribution of a table?
> >
> > J-D
>


Re: Scan across multiple columns

2009-04-09 Thread Ryan Rawson
Check out the org.apache.hadoop.hbase.filter package.  The ColumnValueFilter
might be of help specifically.

The other solution is to do it client side.

-ryan

On Thu, Apr 9, 2009 at 2:45 PM, Vincent Poon (vinpoon) wrote:

> Say I want to scan down a table that looks like this:
>
>Col A  Col B
> row1x x
> row2   x
> row3x x
>
> Normally a scanner would return all three rows, but what's the best way
> to scan so that only row1 and row3 are returned?  i.e. only the rows
> with data in both columns.
>
> Thanks,
> Vincent
>


Re: Scan across multiple columns

2009-04-11 Thread Ryan Rawson
Unless the row is read from disk, how can one know its not the one you want?
This is true for any db system, relational dbs can hide the extra reads
better.

Hbase doesn't provide any query language, so the full cost is realized and
apparent. Server side filters can help reduce network io, but ultimately
you'll need to build secondary indexes if this becomes a primary use case
with high volume. If its analysis, typically people just throw a map reduce
at it and call it a day.

Good luck!

On Apr 11, 2009 9:34 AM, "Lars George"  wrote:

Hi Vincent,

What I did is also have a custom getSplits() implementation in the
TableInputFormat. When the splits are determined I mask out those regions
that have no key of interest. Since the start and end key are ordered as a
total list I can safely assume that if I scan the last few thousand entries
that I can skip the regions beforehand. Of course, if you have a complete
random key or the rows are spread across every region then this is futile.

Lars

Vincent Poon (vinpoon) wrote: > > Thanks for the reply.  I have been using
ColumnValueFilter, but ...


Re: Some HBase FAQ

2009-04-13 Thread Ryan Rawson
HBase loads the index of the files on start-up, if you ran out of memory for
those indexes (which are a fraction of the data size), you'd crash with
OOME.

The index is supposed to be a smallish fraction of the total data size.

I wouldn't run with less than -Xmx2000m

On Mon, Apr 13, 2009 at 10:48 PM, Puri, Aseem wrote:

>
> -Original Message-
> From: Erik Holstad [mailto:erikhols...@gmail.com]
> Sent: Monday, April 13, 2009 9:47 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> On Mon, Apr 13, 2009 at 7:12 AM, Puri, Aseem
> wrote:
>
> > Hi
> >
> >I am new HBase user. I have some doubts regards
> > functionality of HBase. I am working on HBase, things are going fine
> but
> > I am not clear how are things happening. Please help me by answering
> > these questions.
> >
> >
> >
> > 1.  I am inserting data in HBase table and all regions get
> balanced
> > across various Regionservers. But what will happens when data
> increases
> > and there is not enough space in Regionservers to accommodate all
> > regions. So I will like this that some regions in Regionserver and
> some
> > are at HDFS but not on Regionserver or HBase Regioservers stop taking
> > new data?
> >
> Not really sure what you mean here, but if you are asking what to do
> when
> you are
> running out of disk space on the regionservers, the answer is add
> another
> machine
> or two.
>
> --- I want ask that HBase RegionServer store regions data on HDFS. So
> when HBase master starts it loads all region data from HDFS to
> regionserver. So what will the scenario if there is not enough space in
> regionservers to accommodate new data? Is some regions swapped out from
> regionserver to create space for new regions and when needed swaps in
> regions to regionserver from HDFS. Or something else will happen.
>
> >
> >
> >
> > 2.  When I insert data in HBase table, 3 to 4 mapfiles are
> generated
> > for one category, but after some time all mapfiles combines as one
> file.
> > Is this we call minor compaction actually?
> >
> When all current mapfiles and memcache are combined into one files, this
> is called major compaction, see BigTable paper for more details.
>
> >
> >
> >
> > 3.  For my application where I will use HBase will have updates in
> a
> > table frequently. Should is use some other database as a intermediate
> to
> > store data temporarily like MySQL and then do bulk update on HBase or
> > should I directly do updates on HBase. Please tell which technique
> will
> > be more optimized in HBase?
> >
> HBase is fast for reads which has so far been the main focus of the
> development, with
> 0.20 we can hopefully add even fast random reading to it to make it a
> more
> well rounded
> system. Is HBase too slow for you today when writing to it and what are
> your
> requirements?
>
>  Basically I put this question for writing operation. Not any
> complex requirement. I want your suggestion on that what technique
> should I follow for write operation:
>
> a. If there is some update I should store data temporarily in MySQL and
> then do bulk update on HBase
>
> b. As if there is an update I should directly update on HBase instead of
> writing it in MySQL and after some time doing bulk update on HBase.
>
> What you say, what approach is more optimized?
>


Re: Some HBase FAQ

2009-04-13 Thread Ryan Rawson
yes exactly.  The regionserver loads the index on start up in one go, holds
it in ram - then it can use this index to do small specific reads from HDFS.

I found that in hbase 0.20 I was using about 700kB/ram per 5m rows, 40 byte
values.

-ryan

On Mon, Apr 13, 2009 at 11:50 PM, Puri, Aseem wrote:

> Hi Ryan,
>
> It means Regionserver have only index file of regions but not the actual
> data that is on HDFS.
>
> Thanks & Regards
> Aseem Puri
>
> -Original Message-
> From: Ryan Rawson [mailto:ryano...@gmail.com]
> Sent: Tuesday, April 14, 2009 12:16 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> HBase loads the index of the files on start-up, if you ran out of memory
> for
> those indexes (which are a fraction of the data size), you'd crash with
> OOME.
>
> The index is supposed to be a smallish fraction of the total data size.
>
> I wouldn't run with less than -Xmx2000m
>
> On Mon, Apr 13, 2009 at 10:48 PM, Puri, Aseem
> wrote:
>
> >
> > -Original Message-
> > From: Erik Holstad [mailto:erikhols...@gmail.com]
> > Sent: Monday, April 13, 2009 9:47 PM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: Some HBase FAQ
> >
> > On Mon, Apr 13, 2009 at 7:12 AM, Puri, Aseem
> > wrote:
> >
> > > Hi
> > >
> > >I am new HBase user. I have some doubts regards
> > > functionality of HBase. I am working on HBase, things are going fine
> > but
> > > I am not clear how are things happening. Please help me by answering
> > > these questions.
> > >
> > >
> > >
> > > 1.  I am inserting data in HBase table and all regions get
> > balanced
> > > across various Regionservers. But what will happens when data
> > increases
> > > and there is not enough space in Regionservers to accommodate all
> > > regions. So I will like this that some regions in Regionserver and
> > some
> > > are at HDFS but not on Regionserver or HBase Regioservers stop
> taking
> > > new data?
> > >
> > Not really sure what you mean here, but if you are asking what to do
> > when
> > you are
> > running out of disk space on the regionservers, the answer is add
> > another
> > machine
> > or two.
> >
> > --- I want ask that HBase RegionServer store regions data on HDFS. So
> > when HBase master starts it loads all region data from HDFS to
> > regionserver. So what will the scenario if there is not enough space
> in
> > regionservers to accommodate new data? Is some regions swapped out
> from
> > regionserver to create space for new regions and when needed swaps in
> > regions to regionserver from HDFS. Or something else will happen.
> >
> > >
> > >
> > >
> > > 2.  When I insert data in HBase table, 3 to 4 mapfiles are
> > generated
> > > for one category, but after some time all mapfiles combines as one
> > file.
> > > Is this we call minor compaction actually?
> > >
> > When all current mapfiles and memcache are combined into one files,
> this
> > is called major compaction, see BigTable paper for more details.
> >
> > >
> > >
> > >
> > > 3.  For my application where I will use HBase will have updates
> in
> > a
> > > table frequently. Should is use some other database as a
> intermediate
> > to
> > > store data temporarily like MySQL and then do bulk update on HBase
> or
> > > should I directly do updates on HBase. Please tell which technique
> > will
> > > be more optimized in HBase?
> > >
> > HBase is fast for reads which has so far been the main focus of the
> > development, with
> > 0.20 we can hopefully add even fast random reading to it to make it a
> > more
> > well rounded
> > system. Is HBase too slow for you today when writing to it and what
> are
> > your
> > requirements?
> >
> >  Basically I put this question for writing operation. Not any
> > complex requirement. I want your suggestion on that what technique
> > should I follow for write operation:
> >
> > a. If there is some update I should store data temporarily in MySQL
> and
> > then do bulk update on HBase
> >
> > b. As if there is an update I should directly update on HBase instead
> of
> > writing it in MySQL and after some time doing bulk update on HBase.
> >
> > What you say, what approach is more optimized?
> >
>


Re: Some HBase FAQ

2009-04-14 Thread Ryan Rawson
Only a part of the file on HDFS is read into memory to serve the request.
It is not required to hold the entire file in ram.


-ryan

On Mon, Apr 13, 2009 at 11:56 PM, Puri, Aseem wrote:

>
> Ryan,
>
> Thanks for updating me, Also please tell me what will happen if is read
> operation then required region is bring into RAM or not?
>
> Thanks & Regards
> Aseem Puri
>
>
> -----Original Message-
> From: Ryan Rawson [mailto:ryano...@gmail.com]
> Sent: Tuesday, April 14, 2009 12:23 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> yes exactly.  The regionserver loads the index on start up in one go,
> holds
> it in ram - then it can use this index to do small specific reads from
> HDFS.
>
> I found that in hbase 0.20 I was using about 700kB/ram per 5m rows, 40
> byte
> values.
>
> -ryan
>
> On Mon, Apr 13, 2009 at 11:50 PM, Puri, Aseem
> wrote:
>
> > Hi Ryan,
> >
> > It means Regionserver have only index file of regions but not the
> actual
> > data that is on HDFS.
> >
> > Thanks & Regards
> > Aseem Puri
> >
> > -Original Message-
> > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > Sent: Tuesday, April 14, 2009 12:16 PM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: Some HBase FAQ
> >
> > HBase loads the index of the files on start-up, if you ran out of
> memory
> > for
> > those indexes (which are a fraction of the data size), you'd crash
> with
> > OOME.
> >
> > The index is supposed to be a smallish fraction of the total data
> size.
> >
> > I wouldn't run with less than -Xmx2000m
> >
> > On Mon, Apr 13, 2009 at 10:48 PM, Puri, Aseem
> > wrote:
> >
> > >
> > > -Original Message-
> > > From: Erik Holstad [mailto:erikhols...@gmail.com]
> > > Sent: Monday, April 13, 2009 9:47 PM
> > > To: hbase-user@hadoop.apache.org
> > > Subject: Re: Some HBase FAQ
> > >
> > > On Mon, Apr 13, 2009 at 7:12 AM, Puri, Aseem
> > > wrote:
> > >
> > > > Hi
> > > >
> > > >I am new HBase user. I have some doubts regards
> > > > functionality of HBase. I am working on HBase, things are going
> fine
> > > but
> > > > I am not clear how are things happening. Please help me by
> answering
> > > > these questions.
> > > >
> > > >
> > > >
> > > > 1.  I am inserting data in HBase table and all regions get
> > > balanced
> > > > across various Regionservers. But what will happens when data
> > > increases
> > > > and there is not enough space in Regionservers to accommodate all
> > > > regions. So I will like this that some regions in Regionserver and
> > > some
> > > > are at HDFS but not on Regionserver or HBase Regioservers stop
> > taking
> > > > new data?
> > > >
> > > Not really sure what you mean here, but if you are asking what to do
> > > when
> > > you are
> > > running out of disk space on the regionservers, the answer is add
> > > another
> > > machine
> > > or two.
> > >
> > > --- I want ask that HBase RegionServer store regions data on HDFS.
> So
> > > when HBase master starts it loads all region data from HDFS to
> > > regionserver. So what will the scenario if there is not enough space
> > in
> > > regionservers to accommodate new data? Is some regions swapped out
> > from
> > > regionserver to create space for new regions and when needed swaps
> in
> > > regions to regionserver from HDFS. Or something else will happen.
> > >
> > > >
> > > >
> > > >
> > > > 2.  When I insert data in HBase table, 3 to 4 mapfiles are
> > > generated
> > > > for one category, but after some time all mapfiles combines as one
> > > file.
> > > > Is this we call minor compaction actually?
> > > >
> > > When all current mapfiles and memcache are combined into one files,
> > this
> > > is called major compaction, see BigTable paper for more details.
> > >
> > > >
> > > >
> > > >
> > > > 3.  For my application where I will use HBase will have
> updates
> > in
> > > a
> > > > table frequently. Should is use some other database as a
> > intermediate
> > > to
> > > > store data temporarily like My

Re: Some HBase FAQ

2009-04-14 Thread Ryan Rawson
I don't understand the rationale for mysql buffering... HBase handles writes
well, it is not a weak point, so just directly write into HBase.

-ryan

On Tue, Apr 14, 2009 at 12:15 AM, Puri, Aseem wrote:

> So it is possible that without* loading all region we can have some part
> of data in memory that is required.
>
> Can you also suggest me what should I do for a situation:
>
> -- For my application where I will use HBase which will do updates in a
> table frequently. I want your suggestion on that what technique should I
> follow for write operation:
>
> a. If there is some update I should store data temporarily in MySQL and
> then do bulk update on HBase after some time.
>
> Or
>
> b. As if there is an update I should directly update on HBase instead of
> writing it in MySQL.
>
> What you say, what approach is more optimized?
>
> Thanks & Regards
> Aseem Puri
>
> -Original Message-
> From: Ryan Rawson [mailto:ryano...@gmail.com]
> Sent: Tuesday, April 14, 2009 12:33 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> Only a part of the file on HDFS is read into memory to serve the
> request.
> It is not required to hold the entire file in ram.
>
>
> -ryan
>
> On Mon, Apr 13, 2009 at 11:56 PM, Puri, Aseem
> wrote:
>
> >
> > Ryan,
> >
> > Thanks for updating me, Also please tell me what will happen if is
> read
> > operation then required region is bring into RAM or not?
> >
> > Thanks & Regards
> > Aseem Puri
> >
> >
> > -Original Message-
> > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > Sent: Tuesday, April 14, 2009 12:23 PM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: Some HBase FAQ
> >
> > yes exactly.  The regionserver loads the index on start up in one go,
> > holds
> > it in ram - then it can use this index to do small specific reads from
> > HDFS.
> >
> > I found that in hbase 0.20 I was using about 700kB/ram per 5m rows, 40
> > byte
> > values.
> >
> > -ryan
> >
> > On Mon, Apr 13, 2009 at 11:50 PM, Puri, Aseem
> > wrote:
> >
> > > Hi Ryan,
> > >
> > > It means Regionserver have only index file of regions but not the
> > actual
> > > data that is on HDFS.
> > >
> > > Thanks & Regards
> > > Aseem Puri
> > >
> > > -Original Message-
> > > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > > Sent: Tuesday, April 14, 2009 12:16 PM
> > > To: hbase-user@hadoop.apache.org
> > > Subject: Re: Some HBase FAQ
> > >
> > > HBase loads the index of the files on start-up, if you ran out of
> > memory
> > > for
> > > those indexes (which are a fraction of the data size), you'd crash
> > with
> > > OOME.
> > >
> > > The index is supposed to be a smallish fraction of the total data
> > size.
> > >
> > > I wouldn't run with less than -Xmx2000m
> > >
> > > On Mon, Apr 13, 2009 at 10:48 PM, Puri, Aseem
> > > wrote:
> > >
> > > >
> > > > -Original Message-
> > > > From: Erik Holstad [mailto:erikhols...@gmail.com]
> > > > Sent: Monday, April 13, 2009 9:47 PM
> > > > To: hbase-user@hadoop.apache.org
> > > > Subject: Re: Some HBase FAQ
> > > >
> > > > On Mon, Apr 13, 2009 at 7:12 AM, Puri, Aseem
> > > > wrote:
> > > >
> > > > > Hi
> > > > >
> > > > >I am new HBase user. I have some doubts regards
> > > > > functionality of HBase. I am working on HBase, things are going
> > fine
> > > > but
> > > > > I am not clear how are things happening. Please help me by
> > answering
> > > > > these questions.
> > > > >
> > > > >
> > > > >
> > > > > 1.  I am inserting data in HBase table and all regions get
> > > > balanced
> > > > > across various Regionservers. But what will happens when data
> > > > increases
> > > > > and there is not enough space in Regionservers to accommodate
> all
> > > > > regions. So I will like this that some regions in Regionserver
> and
> > > > some
> > > > > are at HDFS but not on Regionserver or HBase Regioservers stop
> > > taking
> > > > > new data?
> > > > >
> > > > Not really sure what you mean here, b

Re: Some HBase FAQ

2009-04-14 Thread Ryan Rawson
The write-ahead log is to recover in crash scenarios.  Even if the
regionserver crashes, recovery from log will save you.

But even so, during a controlled shutdown, regionserver flushes memcache ->
disk.  If the master dies, this flush should get your data to persistent
disk.

On Tue, Apr 14, 2009 at 12:24 AM, Puri, Aseem wrote:

> Actually I read that if HBase master fails cluster will shut down, so I
> think for that instance if the data in currently memcache will also
> lost. So may be it minimize data loss. It's just what I am thinking. If
> I am wrong regarding this issue please correct me.
>
> Thanks & Regards
> Aseem Puri
>
> -Original Message-
> From: Ryan Rawson [mailto:ryano...@gmail.com]
> Sent: Tuesday, April 14, 2009 12:47 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> I don't understand the rationale for mysql buffering... HBase handles
> writes
> well, it is not a weak point, so just directly write into HBase.
>
> -ryan
>
> On Tue, Apr 14, 2009 at 12:15 AM, Puri, Aseem
> wrote:
>
> > So it is possible that without* loading all region we can have some
> part
> > of data in memory that is required.
> >
> > Can you also suggest me what should I do for a situation:
> >
> > -- For my application where I will use HBase which will do updates in
> a
> > table frequently. I want your suggestion on that what technique should
> I
> > follow for write operation:
> >
> > a. If there is some update I should store data temporarily in MySQL
> and
> > then do bulk update on HBase after some time.
> >
> > Or
> >
> > b. As if there is an update I should directly update on HBase instead
> of
> > writing it in MySQL.
> >
> > What you say, what approach is more optimized?
> >
> > Thanks & Regards
> > Aseem Puri
> >
> > -Original Message-
> > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > Sent: Tuesday, April 14, 2009 12:33 PM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: Some HBase FAQ
> >
> > Only a part of the file on HDFS is read into memory to serve the
> > request.
> > It is not required to hold the entire file in ram.
> >
> >
> > -ryan
> >
> > On Mon, Apr 13, 2009 at 11:56 PM, Puri, Aseem
> > wrote:
> >
> > >
> > > Ryan,
> > >
> > > Thanks for updating me, Also please tell me what will happen if is
> > read
> > > operation then required region is bring into RAM or not?
> > >
> > > Thanks & Regards
> > > Aseem Puri
> > >
> > >
> > > -Original Message-
> > > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > > Sent: Tuesday, April 14, 2009 12:23 PM
> > > To: hbase-user@hadoop.apache.org
> > > Subject: Re: Some HBase FAQ
> > >
> > > yes exactly.  The regionserver loads the index on start up in one
> go,
> > > holds
> > > it in ram - then it can use this index to do small specific reads
> from
> > > HDFS.
> > >
> > > I found that in hbase 0.20 I was using about 700kB/ram per 5m rows,
> 40
> > > byte
> > > values.
> > >
> > > -ryan
> > >
> > > On Mon, Apr 13, 2009 at 11:50 PM, Puri, Aseem
> > > wrote:
> > >
> > > > Hi Ryan,
> > > >
> > > > It means Regionserver have only index file of regions but not the
> > > actual
> > > > data that is on HDFS.
> > > >
> > > > Thanks & Regards
> > > > Aseem Puri
> > > >
> > > > -Original Message-
> > > > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > > > Sent: Tuesday, April 14, 2009 12:16 PM
> > > > To: hbase-user@hadoop.apache.org
> > > > Subject: Re: Some HBase FAQ
> > > >
> > > > HBase loads the index of the files on start-up, if you ran out of
> > > memory
> > > > for
> > > > those indexes (which are a fraction of the data size), you'd crash
> > > with
> > > > OOME.
> > > >
> > > > The index is supposed to be a smallish fraction of the total data
> > > size.
> > > >
> > > > I wouldn't run with less than -Xmx2000m
> > > >
> > > > On Mon, Apr 13, 2009 at 10:48 PM, Puri, Aseem
> > > > wrote:
> > > >
> > > > >
> > > > > -Original Message-
> > > > > From: Erik Holstad [mailto:erikhols.

Re: Some HBase FAQ

2009-04-14 Thread Ryan Rawson
every time memcache fills up, you get a flush - that is ~64mb.  But lots of
files reduces performance, so minor compaction just merges them into 1
file.  Since all files are sorted, a file-merge sort is fast and efficient.

this is a minor compaction.  For major compactions one would need to do more
work to prune old values.

-ryan

On Tue, Apr 14, 2009 at 12:46 AM, Puri, Aseem wrote:

> One more thing I want ask that in minor compaction definition in HBase
> documentation is "when the number of MapFiles exceeds a configurable
> threshold, a minor compaction is performed which consolidates the most
> recently written MapFiles"
>
> So it means when we insert data in HBase table, 3 to 4 mapfiles are
> generated for one category, but after some time all mapfiles combines as
> one file. Is this we call minor compaction actually?
>
> Thanks & Regards
> Aseem Puri
>
> -Original Message-
> From: Ryan Rawson [mailto:ryano...@gmail.com]
> Sent: Tuesday, April 14, 2009 12:59 PM
> To: hbase-user@hadoop.apache.org
> Subject: Re: Some HBase FAQ
>
> The write-ahead log is to recover in crash scenarios.  Even if the
> regionserver crashes, recovery from log will save you.
>
> But even so, during a controlled shutdown, regionserver flushes memcache
> ->
> disk.  If the master dies, this flush should get your data to persistent
> disk.
>
> On Tue, Apr 14, 2009 at 12:24 AM, Puri, Aseem
> wrote:
>
> > Actually I read that if HBase master fails cluster will shut down, so
> I
> > think for that instance if the data in currently memcache will also
> > lost. So may be it minimize data loss. It's just what I am thinking.
> If
> > I am wrong regarding this issue please correct me.
> >
> > Thanks & Regards
> > Aseem Puri
> >
> > -Original Message-
> > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > Sent: Tuesday, April 14, 2009 12:47 PM
> > To: hbase-user@hadoop.apache.org
> > Subject: Re: Some HBase FAQ
> >
> > I don't understand the rationale for mysql buffering... HBase handles
> > writes
> > well, it is not a weak point, so just directly write into HBase.
> >
> > -ryan
> >
> > On Tue, Apr 14, 2009 at 12:15 AM, Puri, Aseem
> > wrote:
> >
> > > So it is possible that without* loading all region we can have some
> > part
> > > of data in memory that is required.
> > >
> > > Can you also suggest me what should I do for a situation:
> > >
> > > -- For my application where I will use HBase which will do updates
> in
> > a
> > > table frequently. I want your suggestion on that what technique
> should
> > I
> > > follow for write operation:
> > >
> > > a. If there is some update I should store data temporarily in MySQL
> > and
> > > then do bulk update on HBase after some time.
> > >
> > > Or
> > >
> > > b. As if there is an update I should directly update on HBase
> instead
> > of
> > > writing it in MySQL.
> > >
> > > What you say, what approach is more optimized?
> > >
> > > Thanks & Regards
> > > Aseem Puri
> > >
> > > -Original Message-
> > > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > > Sent: Tuesday, April 14, 2009 12:33 PM
> > > To: hbase-user@hadoop.apache.org
> > > Subject: Re: Some HBase FAQ
> > >
> > > Only a part of the file on HDFS is read into memory to serve the
> > > request.
> > > It is not required to hold the entire file in ram.
> > >
> > >
> > > -ryan
> > >
> > > On Mon, Apr 13, 2009 at 11:56 PM, Puri, Aseem
> > > wrote:
> > >
> > > >
> > > > Ryan,
> > > >
> > > > Thanks for updating me, Also please tell me what will happen if is
> > > read
> > > > operation then required region is bring into RAM or not?
> > > >
> > > > Thanks & Regards
> > > > Aseem Puri
> > > >
> > > >
> > > > -Original Message-
> > > > From: Ryan Rawson [mailto:ryano...@gmail.com]
> > > > Sent: Tuesday, April 14, 2009 12:23 PM
> > > > To: hbase-user@hadoop.apache.org
> > > > Subject: Re: Some HBase FAQ
> > > >
> > > > yes exactly.  The regionserver loads the index on start up in one
> > go,
> > > > holds
> > > > it in ram - then it can use this index to do small specific reads
> > from
> > > > HDFS.
> > 

Re: Basic questions

2009-04-16 Thread Ryan Rawson
Hey,

In HDFS, the namenode contains difficult to replace data in memory and
snapshotted to disk.  The secondary takes copies to a different machine.

In HBase, the master doesn't store anything except transient data, and it
also manages operations.  It can run anywhere, and it will be possible to
have a dual master set up at some point.  The data is stored in HDFS, so
nothing is lost if a master dies.  What exactly will be available in 0.20 I
can't say yet.  The goal at some point is to be fully multi-mastered with
ZooKeeper coordinating our primary-master selection.



On Thu, Apr 16, 2009 at 5:13 PM, Jae Joo  wrote:

> Is "Secondary Name Node" in hadoop can be used in hbase or is it what we
> can
> expect in 0.20?
>
> Jae
>
> On Thu, Apr 16, 2009 at 7:38 PM, Jae Joo  wrote:
>
> > I have read and foind that the master node is SPOF and 0.20 will have
> > failover system for master node. Any update or any solution to avio
> > SPOF?
> > Jae
> >
> > On 4/16/09, Billy Pearson  wrote:
> > > hbase and hadoop both have a single node (namenode & master) that tell
> > the
> > > clients where stuff is that's all they do is keep up with
> > > where stuff is they do not handle the getting of the data the client
> api
> > > will go to the node with the data they tell it to find the data and get
> > the
> > > data.
> > >
> > > as for adding new nodes to hadoop to balance the cluster you can use
> > > bin/start-balancer.sh -threshold 5
> > > Where 5 is the % you want the nodes to be with in each other in terms
> of
> > > used %
> > >
> > > Billy
> > >
> > >
> > > "Jae Joo"  wrote in
> > > message 
> > > news:3eb24ef50904161430we261882v12600704f521...@mail.gmail.com
> .
> > ..
> > >> Hi,
> > >>
> > >> We are doing some feasibility study of HBase and have questions.
> > >> 1. Once we have build tables on certain number node and decide to add
> > more
> > >> nodes, how it is working? I know that it is not hbase, but hadoop
> > >> questions.
> > >> Will it automatically redistribute the files or start utilizing the
> new
> > >> node
> > >> for new inserted records?
> > >>
> > >> 2. Is there any way to have multile master nodes? Should all traffis
> (or
> > >> requests) be going to SIngle Master node to access the data from
> hbase?
> > >>
> > >>
> > >> Thanks,
> > >>
> > >> Jae
> > >>
> > >
> > >
> > >
> >
> > --
> > Sent from my mobile device
> >
>


Re: Replicating data into HBase

2009-04-17 Thread Ryan Rawson
HBase is not a relational database, so many things that are in a SQL
database dont exist.

eg:
- sequences
- secondary declarative keys
- joins
- advance query features such as order by, group by
- operators of any kind

Given conventions (eg: naming of index tables), it might be possible to
semi-automatedly convert data, but it might not efficiently take advantage
of HBase's unique schema-less design.

I suggest you have a look at the Google's bigtable paper, as it has the same
underlying model that HBase does.

Good luck!


On Fri, Apr 17, 2009 at 3:30 PM, Brian Forney  wrote:

> Hi all,
>
> I'd like to replicate a large dataset from a relational database into HBase
> for better throughput of MapReduce jobs. Has anyone had success replicating
> from a relational database (in my case SQL Server) to HBase?
>
> Thanks,
> Brian
>


Randomize your input during huge imports

2009-04-21 Thread Ryan Rawson
Here is the MapReduce I use to randomize the lines of a file.  I've omitted
the imports for brevity - your IDE can fix that.

Enjoy!
-ryan

public class Randomize {

  // technically text/text could be 'object'.
  public static class Map extends MapReduceBase implements
Mapper {
Random rnd = new Random();

public void map(LongWritable key, Text value,
OutputCollector output, Reporter reporter)
throws IOException {
  IntWritable redKey = new IntWritable(rnd.nextInt(10));
  output.collect(redKey, value);
  reporter.setStatus("Map emitting cell for: " + redKey);

}

  }

  // This combiner reduces the time of a map-reduce from 1h18m -> 48m.
  // That is a 38% improvement (!!).
  public static class Combiner extends MapReduceBase
  implements Reducer {

public void reduce(IntWritable key, Iterator values,
OutputCollector output, Reporter reporter)
throws IOException {
  Text out = new Text();
  byte newline [] = {'\n'};
  int siz = 0;
  while (values.hasNext())
  {
Text txt = values.next();
out.append(txt.getBytes(), 0, txt.getLength());

if (++siz > 500) {
  output.collect(key, out);
  siz = 0;
  out = new Text();
} else {
  if (values.hasNext())
out.append(newline, 0, newline.length);
}
  }
  output.collect(key, out);
}
  }

  public static class Reduce extends MapReduceBase implements
Reducer {
public void reduce(IntWritable key, Iterator values,
OutputCollector output, Reporter reporter)
throws IOException {
  while (values.hasNext())
  {
output.collect(NullWritable.get(), values.next());
  }
}

  }
  public static void main(String [] argv) throws IOException {
if (argv.length < 2) {
  System.out.println("Usage:  ");
  return;
}
JobConf job = new JobConf(Randomize.class);
job.setJobName("Randomize: " + argv[0]);
FileInputFormat.setInputPaths(job, new Path(argv[0]));
job.setInputFormat(TextInputFormat.class);
job.setMapperClass(Map.class);
job.setReducerClass(Reduce.class);
job.setCombinerClass(Combiner.class);
FileOutputFormat.setOutputPath(job, new Path(argv[1]));
job.setOutputFormat(TextOutputFormat.class);

job.setOutputKeyClass(IntWritable.class);
job.setOutputValueClass(Text.class);

JobClient jc = new JobClient(job);
jc.submitJob(job);
  }
}


Re: Crawling Using HBase as a back end --Issue

2009-04-25 Thread Ryan Rawson
Man I really understand the frustration when something just _does_not_work_,
especially if advertised to work so.

But the thing to remember here, is hbase is cutting edge database stuff -
highly clustered distributed databases are not super straighforward.  For an
example, take Oracle RAC - you won't be able to get that up and running at
any interesting performance levels without paying oracle or a highly
experienced oracle dba to tune it just right.

So, considering how few tuning parameters, and how scalable hbase is, I
think it's a great deal for the price.

On Sat, Apr 25, 2009 at 8:19 PM, Andrew Purtell  wrote:

>
> Right, well "hbase did not work" with no details as to why
> does not help us to improve it. Please kindly consider
> asking your colleague to forward details at your and/or
> that person's convenience. Also, for future reference, HBase
> has a responsive developer community and could have likely
> helped for only the cost of time to file a bug report and
> respond to inquiries for more information.
>
>   - Andy
>
> > From: Derek Pappas
> > Subject: Re: Crawling Using HBase as a back end --Issue
> > Date: Thursday, April 23, 2009, 11:35 PM
> > Someone else in the company knows the details. Sorry did not
> > mean to pan hbase. We are a very small startup and needed to
> > get a prototype (version 2) working. We tried using hbase
> > back in the Dec/Jan time frame.
>
>
>
>
>


Re: Monitoring HBase

2009-04-26 Thread Ryan Rawson
Have a look at conf/hadoop-metrics.properties - hbase can publish ganglia
metrics which you can then do whatever with.  That is one good way to assess
the operation of your cluster.

Good luck,
-ryan

On Sun, Apr 26, 2009 at 12:46 AM, Genady  wrote:

> Hi,
>
> 1.What could be a best way to monitor HBase, not only that region servers
> is
> online, but also is serving regions?
> 2.In case we choose just to parse regionserver.jsp returned HTML, is there
> is way to add a basic authentication login dialog?
>
> Thanks for any help,
> Gennady
>
>


Re: Force split

2009-04-29 Thread Ryan Rawson
It works, it has worked in the past...

One thing to remember is you can't always split...  You cant split until all
the open scanners are closed.  Furthermore you can't split if a region has
just split.  The reason for this is because once you split a region, files
aren't rewritten, but the 2 daughter regions hold references to the parent
region.  Until those regions are resolved via compaction (slow, minutes
possibly), you can't split those daughter regions.

So check again, maybe your splits are working after all.

If you are doing imports, check out my Randomize.java i posted to the list
last week.  That helps import performance substantially.

On Wed, Apr 29, 2009 at 12:52 AM, Edward J. Yoon wrote:

> Hi,
>
> I saw that function on the web UI but it seems not implemented yet.
>
> public void modifyTable(final byte[] tableName, int op, Writable[] args)
>  switch (op) {
>...
>case MODIFY_TABLE_SPLIT:
>case MODIFY_TABLE_COMPACT:
>..
>  }
> }
>
> Am I right? Pls let me know if i'm wrong.
> --
> Best Regards, Edward J. Yoon @ NHN, corp.
> edwardy...@apache.org
> http://blog.udanax.org
>


Re: Improving import performance

2009-04-29 Thread Ryan Rawson
You might have to delve into tweaking the GC settings.  Here is what I am
setting:

export HBASE_OPTS="-XX:+UseConcMarkSweepGC -XX:NewSize=12m
-XX:MaxNewSize=12m -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
-Xloggc:/export/hadoop/logs/gc.log"

What I found is once your in-memory complexity rises, the JVM resizes the
ParNew/new-gen space up and up, thus extending the length of the so-called
'minor collections'.  At one point with a 150MB ParNew I was seeing
typically 100-150ms and outliers of 400ms for 'minor' GC pauses.  If you
have all machines in your cluster pausing for 100ms at semi-random
intervals, that holds up your import as the clients are waiting on the
paused JVM to continue.

The key thing I had to do was set -XX:MaxNewSize=12m (about 2 * my L2 on
xeons).  You get more frequent, but smaller, GCs.  Your heap also tends to
grow larger than before, but with CMS it doesnt result in larger VM pauses,
just more ram usage.  I I personally use -Xmx4500m.   With your machines,
I'd consider a setting at least 1500, preferably 2000-2500m.  Of course I
have a ton of heap to chuck at it, so CMS collections dont happen all the
time (but when they do, they can prune 1500mb of garbage).

Since you are on a 2 core, you will probably have to set the CMS to
incremental:
-XX:+CMSIncrementalMode

To prevent the CMS GC from starving out your main threads.

Good luck with it!
-ryan

On Wed, Apr 29, 2009 at 3:33 PM, Jim Twensky  wrote:

> Hi,
>
> I'm doing some experiments to import large datasets to Hbase using a Map
> job. Before posting some numbers, here is a summary of my test cluster:
>
> I have 7 regionservers and 1 master. I also run HDFS datanodes and Hadoop
> tasktrackers on the same 7 regionservers. Similarly, I run the Hadoop
> namenode on the same machine that I run the Hbase master. Each machine is
> an
> IBM e325 node that has two 2.2 GHz AMD64 processors, 4 GB RAM, and 80 GB
> local disk.
>
> My dataset is simply the output of another map reduce job, consisting of 7
> sequence files with a total size of 40 GB. Each file contains key, value
> records of the form (Text, LongWritable). The keys are sentences or phrases
> extracted from sentences and the values are frequencies. The total number
> of
> records is roughly 420m and an average key is around 100 bytes. (40GB /
> 420m
> - ignoring long writables)
>
> I tried to randomize the (key,value) pairs with another map reduce job and
> I
> also set:
>
>table.setAutoFlush(false);
>table.setWriteBufferSize(1024*1024*10);
>
> based on some advice that I read before on the list. My Map function that
> imports data to Hbase is as follow:
>
>public void map(Text key, LongWritable value,
> OutputCollector output, Reporter reporter)
> throws IOException {
>
>BatchUpdate update = new BatchUpdate(key.toString());
>update.put("frequency:value",Bytes.toBytes(value.get()));
>
>table.commit(update);
>}
>
> So far I can hit 20% of the import in 40-45 minutes so importing the whole
> data set will presumbly take more than 3.5 hours. I tried diffirent write
> buffer sizes between5 MB and 20 MB and didn't get any significant
> improvements. I did my experiments with 1 or 2 mappers per node although 1
> mapper per node seemed to do better than 2 nodes.  When I refresh the Hbase
> master web interface during my imports, I see the requests are generally
> divided equally to 7 regionservers and as I keep hitting the refreh button,
> I can see that I get 1 to 7 requests at once.
>
> I read some earlier posts from Ryan and Stack, and I was actually expecting
> at least twice better performonce so I decided to ask to the list whether
> this is an expected performance or way below it.
>
> I'd appreciate any comments/suggestions.
>
>
> Thanks,
> Jim
>


Re: Performance of hbase importing

2009-04-29 Thread Ryan Rawson
Hey,

I wrote a reply to a different thread which encapsulates most my recent
learning and understanding of how GC and the JVM impacts large scale data
import.

At this point, I have a 19 machine cluster, with 30 TB of aggregate storage
on raid0 (2 disks/box).   I've devoted them to hbase 0.20 testing, and I've
been able to load a massive set of (real) data in.  Unlike previous data
sets, this one is both (a) huge and (b) tiny rows.

One thing I am finding is I end up with weird bottlenecks:
- The clients dont always seem to be able to push maximal speed
- GC pauses are death
- Compaction thread limit might be holding things up, but I'm not sure about
this one yet.
- In-memory complexity and size is pretty much stressing the JVM
significantly.

The bottom line is we are fighting against the JVM now - both with GC
problems, and as well as general efficiency.  For example, a typical
regionserver can carry a memcache load of 1000-1500m.  That is a lot of
outstanding writes.

As for numbers, I generally want to see the following import performance to
be happy:
- 100-130k ops/sec across 19 nodes
- 125-200 MB/sec of network traffic across all nodes
- 76 map reads reading from mysql -> hbase

This is currently sustainable with 3k regions for prolonged periods of
time.  I have an import that has run for 12 hours at these speeds.

Speed problems start to manifest themselves as dips in the network
performance graph.  The bigger dips (when I was having maximal GC pause
problems) would bounce performance between 0 and 175MB/sec.  Smaller ones
could be due to io-wait or other inefficiencies.

It's all about the GC pause!

-ryan

On Wed, Apr 29, 2009 at 2:56 PM, Jim Twensky  wrote:

> Hi Ryan,
>
> Have you got your new hardware? I was keeping an eye on your blog for the
> past few days but I haven't seen any updates there so I just decided to ask
> you on the list. If you have some results, would you like to give us some
> numbers along with hardware details?
>
> Thanks,
> Jim
>
> On Thu, Jan 15, 2009 at 2:28 PM, Larry Compton
> wrote:
>
> > That explains it. Thanks!
> >
> > On Thu, Jan 15, 2009 at 2:11 PM, Jean-Daniel Cryans  > >wrote:
> >
> > > Larry,
> > >
> > > This feature was done for 0.19.0 for which a release candidate is on
> the
> > > way.
> > >
> > > J-D
> > >
> > > On Thu, Jan 15, 2009 at 2:03 PM, Larry Compton
> > > wrote:
> > >
> > > > I'm interested in trying this, but I'm not seeing "setAutoFlush()"
> and
> > > > "setWriteBufferSize()" in the "HTable" API (I'm using HBase 0.18.1).
> > > >
> > > > Larry
> > > >
> > > > On Sun, Jan 11, 2009 at 5:11 PM, Ryan Rawson 
> > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > New user of hbase here. I've been trolling about in IRC for a few
> > days,
> > > > and
> > > > > been getting great help all around so far.
> > > > >
> > > > > The topic turns to importing data into hbase - I have largeish
> > datasets
> > > I
> > > > > want to evaluate hbase performance on, so I've been working at
> > > importing
> > > > > said data.  I've managed to get some impressive performance
> speedups,
> > > and
> > > > I
> > > > > chronicled them here:
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html
> > > > >
> > > > > To summarize:
> > > > > - Use the Native HBASE API in Java or Jython (or presumably any JVM
> > > > > language)
> > > > > - Disable table auto flush, set write buffer large (12M for me)
> > > > >
> > > > > At this point I can import a 18 GB, 440m row comma-seperated flat
> > file
> > > in
> > > > > about 72 minutes using map-reduce.  This is on a 3 node cluster all
> > > > running
> > > > > hdfs,hbase,mapred with 12 map tasks (4 per).  This hardware is
> loaner
> > > DB
> > > > > hardware, so once I get my real cluster I'll revise/publish new
> data.
> > > > >
> > > > > I look forward to meeting some of you next week at the hbase meetup
> > at
> > > > > powerset!
> > > > >
> > > > > -ryan
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Larry Compton
> > SRA International
> > 240.373.5312 (APL)
> > 443.742.2762 (cell)
> >
>


Re: Problems when executing many (?) HTable.lockRow()

2009-05-08 Thread Ryan Rawson
Hey,

You can bundle any number of operations (put and delete) to 1 row with
BatchUpdate thus obviating the need for using explicit row locks (the server
does the locks internally as well, so in this case you are locking twice).

-ryan

On Fri, May 8, 2009 at 4:29 PM, Guilherme Germoglio wrote:

> Hello,
>
> I've been running some tests using HTable.lockRow but I think it is not
> performing well. One of my tests is quite simple: run several threads that
> will execute the following lines for the same row:
>
>   RowLock rowLock = htable.lockRow(Bytes.toBytes(row));
>   htable.commit(batchupdate, rowLock);
>   htable.unlockRow(rowLock);
>
> where the batchupdate only does a single "put" on a single column.
>
> The problem is that it is running very very slow when I use only 20
> threads:
> about 30 minutes against the 6~10 seconds of running with 10 threads. Also,
> when using 20 threads, some *java.io.IOException: java.io.IOException:
> Invalid row lock* messages are printed.
>
> Since it is the first time I'm using RowLocks, I'm not sure if the problem
> is in my test or in HBase. So, I'm asking you for help.
>
> I'm using hbase 0.19.2 (rc) in pseudo-distributed mode on a mac book, but
> the behavior is the same when using 0.19.0. Logs, hbase-site.xml (which is
> the default) and the test can be found on the following link:
>
> http://germoglio.googlepages.com/logs.zip
>
> Please notice that although this test doesn't make much sense of locking,
> the idea is to evolve it in order to perform a few operations besides the
> only single put between lockRow and unlockRow methods.
>
> Thank you very much,
>
> --
> Guilherme
>
> msn: guigermog...@hotmail.com
> homepage: http://germoglio.googlepages.com
>


Re: Problems when executing many (?) HTable.lockRow()

2009-05-08 Thread Ryan Rawson
There is no way that this kind of action would ever be performant, with 4
rpc calls.

Have a look at the 'atomicIncrement' call...

-ryan


On Fri, May 8, 2009 at 6:57 PM, Guilherme Germoglio wrote:

> thank you, Ryan.
>
> what about changing a row value to something that depends on its previous
> value? An atomic increment of an integer, for example. I'm thinking
> something like the following (in pseudocode - sort of :-):
>
> lock = htable.lockRow(row);
> value = htable.getRow(row, lock);
> value = doSomething(value);
> batchupdate.put(row, column, value);
> htable.commit(batchupdate, lock);
> htable.unlockRow(lock);
>
> On Fri, May 8, 2009 at 8:51 PM, Ryan Rawson  wrote:
>
> > Hey,
> >
> > You can bundle any number of operations (put and delete) to 1 row with
> > BatchUpdate thus obviating the need for using explicit row locks (the
> > server
> > does the locks internally as well, so in this case you are locking
> twice).
> >
> > -ryan
> >
> > On Fri, May 8, 2009 at 4:29 PM, Guilherme Germoglio  > >wrote:
> >
> > > Hello,
> > >
> > > I've been running some tests using HTable.lockRow but I think it is not
> > > performing well. One of my tests is quite simple: run several threads
> > that
> > > will execute the following lines for the same row:
> > >
> > >   RowLock rowLock = htable.lockRow(Bytes.toBytes(row));
> > >   htable.commit(batchupdate, rowLock);
> > >   htable.unlockRow(rowLock);
> > >
> > > where the batchupdate only does a single "put" on a single column.
> > >
> > > The problem is that it is running very very slow when I use only 20
> > > threads:
> > > about 30 minutes against the 6~10 seconds of running with 10 threads.
> > Also,
> > > when using 20 threads, some *java.io.IOException: java.io.IOException:
> > > Invalid row lock* messages are printed.
> > >
> > > Since it is the first time I'm using RowLocks, I'm not sure if the
> > problem
> > > is in my test or in HBase. So, I'm asking you for help.
> > >
> > > I'm using hbase 0.19.2 (rc) in pseudo-distributed mode on a mac book,
> but
> > > the behavior is the same when using 0.19.0. Logs, hbase-site.xml (which
> > is
> > > the default) and the test can be found on the following link:
> > >
> > > http://germoglio.googlepages.com/logs.zip
> > >
> > > Please notice that although this test doesn't make much sense of
> locking,
> > > the idea is to evolve it in order to perform a few operations besides
> the
> > > only single put between lockRow and unlockRow methods.
> > >
> > > Thank you very much,
> > >
> > > --
> > > Guilherme
> > >
> > > msn: guigermog...@hotmail.com
> > > homepage: http://germoglio.googlepages.com
> > >
> >
>
>
>
> --
> Guilherme
>
> msn: guigermog...@hotmail.com
> homepage: http://germoglio.googlepages.com
>


Re: Problems when executing many (?) HTable.lockRow()

2009-05-09 Thread Ryan Rawson
Hey Guilherme,

We'd need more logs from the regionserver affected to know more.  "unable to
contact regionserver" is a generic client error.  Consider running your
machines under debug logging.

Also, I don't think you should be calling lockRow/unlockRow - if you are
attempting to get an atomic commit against a row, you don't need explicit
locking.  Pseudo code wise one might be tempted to do this:

lockRow(row)
bu = new BatchUpdate(row)
bu.delete("col")
table.commit(bu)

bu = new BatchUpdate(row)
bu.put("foo")
table.commit(bu)
unlockRow(row)


but one should really write this instead:
bu = new BatchUpdate(row)
bu.delete("col")
bu.put("col")
table.commit(bu)

Internally to the Regionserver, it locks the row and unlocks it.  I would
never expect performance out of explicit row locks.

On Sat, May 9, 2009 at 3:46 PM, Guilherme Germoglio wrote:

> Hello,
>
> I have made some measurements and I think the problem is more serious than
> poor performance. Many threads are failing to commit or to unlock the row
> when running my test with 20 threads. However, when tuning what to print
> and
> measure, I run twice (or thrice, I'm not sure) and none failed.
>
> The data from these runs are here:
> http://spreadsheets.google.com/pub?key=rZwAh62-rx3Yvk1W2VtHFUg
>
> The numbers are in milliseconds and the negative indicates that the
> operation was not performed.
>
> Thanks,
>
>
> On Fri, May 8, 2009 at 11:35 PM, Ryan Rawson  wrote:
>
> > There is no way that this kind of action would ever be performant, with 4
> > rpc calls.
> >
> > Have a look at the 'atomicIncrement' call...
> >
> > -ryan
> >
> >
> > On Fri, May 8, 2009 at 6:57 PM, Guilherme Germoglio  > >wrote:
> >
> > > thank you, Ryan.
> > >
> > > what about changing a row value to something that depends on its
> previous
> > > value? An atomic increment of an integer, for example. I'm thinking
> > > something like the following (in pseudocode - sort of :-):
> > >
> > > lock = htable.lockRow(row);
> > > value = htable.getRow(row, lock);
> > > value = doSomething(value);
> > > batchupdate.put(row, column, value);
> > > htable.commit(batchupdate, lock);
> > > htable.unlockRow(lock);
> > >
> > > On Fri, May 8, 2009 at 8:51 PM, Ryan Rawson 
> wrote:
> > >
> > > > Hey,
> > > >
> > > > You can bundle any number of operations (put and delete) to 1 row
> with
> > > > BatchUpdate thus obviating the need for using explicit row locks (the
> > > > server
> > > > does the locks internally as well, so in this case you are locking
> > > twice).
> > > >
> > > > -ryan
> > > >
> > > > On Fri, May 8, 2009 at 4:29 PM, Guilherme Germoglio <
> > germog...@gmail.com
> > > > >wrote:
> > > >
> > > > > Hello,
> > > > >
> > > > > I've been running some tests using HTable.lockRow but I think it is
> > not
> > > > > performing well. One of my tests is quite simple: run several
> threads
> > > > that
> > > > > will execute the following lines for the same row:
> > > > >
> > > > >   RowLock rowLock = htable.lockRow(Bytes.toBytes(row));
> > > > >   htable.commit(batchupdate, rowLock);
> > > > >   htable.unlockRow(rowLock);
> > > > >
> > > > > where the batchupdate only does a single "put" on a single column.
> > > > >
> > > > > The problem is that it is running very very slow when I use only 20
> > > > > threads:
> > > > > about 30 minutes against the 6~10 seconds of running with 10
> threads.
> > > > Also,
> > > > > when using 20 threads, some *java.io.IOException:
> > java.io.IOException:
> > > > > Invalid row lock* messages are printed.
> > > > >
> > > > > Since it is the first time I'm using RowLocks, I'm not sure if the
> > > > problem
> > > > > is in my test or in HBase. So, I'm asking you for help.
> > > > >
> > > > > I'm using hbase 0.19.2 (rc) in pseudo-distributed mode on a mac
> book,
> > > but
> > > > > the behavior is the same when using 0.19.0. Logs, hbase-site.xml
> > (which
> > > > is
> > > > > the default) and the test can be found on the following link:
> > > > >
> > > > > http://germoglio.googlepages.com/logs.zip
> > > > >
> > > > > Please notice that although this test doesn't make much sense of
> > > locking,
> > > > > the idea is to evolve it in order to perform a few operations
> besides
> > > the
> > > > > only single put between lockRow and unlockRow methods.
> > > > >
> > > > > Thank you very much,
> > > > >
> > > > > --
> > > > > Guilherme
> > > > >
> > > > > msn: guigermog...@hotmail.com
> > > > > homepage: http://germoglio.googlepages.com
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Guilherme
> > >
> > > msn: guigermog...@hotmail.com
> > > homepage: http://germoglio.googlepages.com
> > >
> >
>
>
>
> --
> Guilherme
>
> msn: guigermog...@hotmail.com
> homepage: http://germoglio.googlepages.com
>


Re: HTTP Error 404 for HBase webapp

2009-05-09 Thread Ryan Rawson
I generally rsync the entire hbase dir including "build" after doing just
"ant".  Sounds like the web jazz isn't being included somehow.  I confess
I'm fairly weak on this aside from knowing how to edit jsp.

On May 9, 2009 4:22 PM, "Tom Santos"  wrote:

I'm running hadoop 0.20.0 and I used svn to get the hbase trunk as of today
(May 9).  I did an 'ant package' in the hbase trunk, configured it for 5
nodes, distributed it to them, and started up my hbase.  It seems to make a
valid-looking /hbase dir in my DFS and the hmaster and regionservers come up
with no errors but the webapp running on 60010 just says:

HTTP ERROR: 404

/webapps/master/master.jsp

RequestURI=/webapps/master/master.jsp

Powered by Jetty://

The regionserver webapps seem to have the same problem.

Is it something I did with my build?  It seemed like 'ant package' generated
a good looking directory but I can't be absolutely sure.

Is there a config item that's new (since 19.1) that makes it so the webapp
will work?

Thanks for any insight you may have,

--
Tom Santos
santos@gmail.com


Re: Problems when executing many (?) HTable.lockRow()

2009-05-09 Thread Ryan Rawson
if we had a compare and set operation would this help you?

Another thing, normally hbase regionserver only runs 10 ipc handler
threads... I'm not sure how relevant or important this is yet. Holding a row
open should not tie up an ipc thread though.

On May 9, 2009 4:19 PM, "Guilherme Germoglio"  wrote:

Hey Ryan,

Thank for such a quick response!

The log is here: http://germoglio.googlepages.com/log.zip Unfortunately, I
wasn't running with debug on. I'll do it tomorrow in order to get more data
(Will I be able to find how to turn on debug in the mailing list archives or
in hbase wiki?). Also, you will notice that the log is only from the master
server. The reason for this is that I'm running on the pseudodistributed
mode and the region server log has only one message "2009-05-09 19:04:26,242
WARN org.apache.hadoop.hbase.regionserver.HRegionServer: Not starting a
distinct region server because hbase.master is set to 'local' mode".

Thank you for the advice. I understand that a simple put doesn't make sense
to lock a row. However, I'm just considering the performance of it. If the
put operation into a single row (which I think is the fastest operation)
isn't fast enough for my project, I'll not even try what I'm really
intending to do, which is atomically getting a value, modifying it (not
necessarily incrementing it), and then putting it back to the table.

I wasn't expecting great performance either, but is it ok to execute a
single put into a locked row in 5~10 minutes?

On Sat, May 9, 2009 at 7:54 PM, Ryan Rawson  wrote: >
Hey Guilherme, > > We'd ...
--

Guilherme msn: guigermog...@hotmail.com homepage:
http://germoglio.googlepages.com


Re: Problems when executing many (?) HTable.lockRow()

2009-05-09 Thread Ryan Rawson
You have to think if this is a realistic load scenario, will you really have
200 threads executing as fast as possible per hbase instance? Remember you
realistically need a 10 node cluster.

Even with a high writer count will you really be writing back to back?

Also think about your mutation semantics, with the history and multi column
features hbase gives you, are you sure you'll need to do what you're doing?
Hbase is not sql, and take advantage of the features we provide.

On May 9, 2009 4:48 PM, "Guilherme Germoglio"  wrote:

But with 200 threads, the problem is back. =\

On Sat, May 9, 2009 at 8:44 PM, Guilherme Germoglio wrote:

> maybe you've just spotted the problem! I've raised >
hbase.regionserver.handler.count to 100 and ...
--

Guilherme msn: guigermog...@hotmail.com homepage:
http://germoglio.googlepages.com


Re: Web BI with HBase : Use Case

2009-05-11 Thread Ryan Rawson
Hey,

There has huge improvements in latency since 0.19, in the current
trunk/0.20-dev work going on.  There is plans afoot to hopefully have a
release by June.

I am seeing reasonable latency numbers from HBase, but my case is very
pathalogical - small cells, and many of them (1k+).  Given the performance
characteristics and what was fixed in 0.20, I hope the new stuff will be
fast enough for your needs.

Furthermore, we have LZO compression enabled reliabily in 0.20, this will
help storing this kind of thing greatly with not much overhead during read
time.

Generally though, this kind of area should be something hbase should shine
at.

If you are willing to put up with risk, you can try the latest svn trunk.
There are still big pending changes, so dont count on your /hbase being able
to be carried through to the actual 0.20 release, but give it a shot.

Thanks,
-ryan

On Mon, May 11, 2009 at 3:56 PM, dotnetmetal  wrote:

> Hey there,
>
> I've been working with Hadoop for about a year now, and have recently been
> tasked with our new metadata storange and analysis platform. I'm looking
> for
> your advice into what I should research, and if HBase is right for our use
> cases.
>
> Currently, we're collecting documents onto our Hadoop cluster, and then
> indexing them with Lucene (and Katta). Documents have attributes like a
> create date, author, bodytext, domain, etc.
>
> We're looking at 20TB of data to start with, growing by a few dozen a day.
>
> I'm researching the best way to provide BI on top of this data that our
> customers can "Slice and Dice" on. HBase has some appealing
> characteristics,
> but I'm not sure if it's *quite* what we need, since latency is an issue.
> Lucene has great indexing, but we're also going to be adding metadata
> constantly and performing schema changes.
>
> Here's a use case:
>
> A customer searches for a keyword in our web UI and a list of a few hundred
> thousand documents is returned. The customer would then like to select a
> few
> random authors from those documents for a certain date range (let's say 4
> months), and get a count of documents per author. A few hours later, these
> documents are tagged with some more metadata... say, PageRank of the parent
> domain. The user can use this data as part of his queries as well. We'd
> like
> to have a response time of 10 seconds or so.
>
> I don't care much about storage space, so denormalization is totally fine.
> Is this a problem we can tackle in HBase or another open source distributed
> DB?
>
> A company called "Vertica" claims to be able to do this, but I wasn't very
> impressed with their architecture. "Greenplum" also looks interesting, but
> I
> haven't researched them much yet.
>
> Thanks for all your help!
>


Re: Hbase 0.19.2 - Large import results in heavily unbalanced hadoop DFS

2009-05-13 Thread Ryan Rawson
I get this as well... It seems to be due to delayed block deletion in HDFS.
Once the cluster settles down for a while, the blocks get deleted and we're
back in balance.

On Wed, May 13, 2009 at 10:56 AM, Alexandra Alecu  wrote:

>
> I am a Hbase/hadoop beginner.
> As an initial test, I am trying to import about 120GB of records into one
> big table in HBase (replication level 2).  I have a HBase master and a
> Hadoop namenode running on two separate machines and 4 other nodes running
> the datanodes and regionservers. Each datanode has approx 400 GB local
> storage.
>
> I have done a few tests previously with Hbase 0.19.1 and I kept on running
> into problems related to the slow compactions (HBASE-1058).  I have now
> installed HBase 0.19.2 and one thing I noticed is that the disk usage
> during
> import is much higher and the datanodes come out very unbalanced.
>
> Whereas using HBase 0.19.1, I used to fill about 300 GB nicely balanced,
> now
> I have filled about 700GB, 100GB on each of the 3 datanodes and one of the
> nodes gets completely full (400GB) causing the import to slowdown and
> eventually fail not being able to contact one of the .META. regions.
>
> I stopped HBase and tried to balance the hdfs which informed me :
>
> 09/05/13 17:34:38 INFO balancer.Balancer: Need to move 177.92 GB bytes to
> make the cluster balanced.
>
> After this, with hadoop hard at work balancing, it seems to fail to move
> blocks 50% of the time, should I worry about these errors/warnings:
> 09/05/13 17:34:38 WARN balancer.Balancer: Error moving block
> -6198018159178133648 from 131.111.70.215:50010 to 131.111.70.214:50010
> through 131.111.70.216:50010: block move is failed
>
> Checking the balancing process, it looks like the hdfs usage constantly
> decreases, having at the end a value closer to what i expected.
> Essentially,
> it looks like the balancing has wiped the data which was causing this one
> datanode to fill up to almost 100%. Maybe this data was caused by the
> delayed compaction or some logs which need to be played on the cluster.
>
> This is the situation towards the end of the balancing :
>
> Datanodes available: 4 (4 total, 0 dead)
>
> Name: 1
> Configured Capacity: 433309891584 (403.55 GB)
> DFS Used: 88593623040 (82.51 GB)
> DFS Used%: 20.45%
> DFS Remaining%: 78.51%
> Last contact: Wed May 13 18:48:09 BST 2009
>
> Name: 2
> Configured Capacity: 433309891584 (403.55 GB)
> DFS Used: 89317653511 (83.18 GB)
> DFS Used%: 20.61%
> DFS Remaining%: 78.34%
> Last contact: Wed May 13 18:48:10 BST 2009
>
> Name: 3
> Configured Capacity: 433309891584 (403.55 GB)
> DFS Used: 89644974080 (83.49 GB)
> DFS Used%: 20.69%
> DFS Remaining%: 78.27%
> Last contact: Wed May 13 18:48:10 BST 2009
>
> Name: 4
> Configured Capacity: 433309891584 (403.55 GB)
> DFS Used: 138044233537 (128.56 GB)
> DFS Used%: 31.86%
> DFS Remaining%: 67.07%
> Last contact: Wed May 13 18:48:10 BST 2009
>
> Before the balancing, the datanode no 4 was using approx 400 GB.
>
> What are your comments on this behaviour?  Is this something that you
> expected?
>
> Let me know if you need me to provide more information.
>
> Many thanks,
> Alexandra Alecu.
>
>
>
> --
> View this message in context:
> http://www.nabble.com/Hbase-0.19.2---Large-import-results-in-heavily-unbalanced-hadoop-DFS-tp23526652p23526652.html
> Sent from the HBase User mailing list archive at Nabble.com.
>
>


Re: Administration tool for HBase

2009-05-13 Thread Ryan Rawson
There are tools:

bin/start-hbase.sh
bin/stop-hbase.sh

starts/stops cluster.

per node:
bin/hbase-daemon.sh start regionserver
bin/hbase-daemon.sh stop regionserver

you might need HBASE-1362 however.

On Wed, May 13, 2009 at 11:46 PM, Ninad Raut wrote:

> Hi,
> I have been asekd to automate the following processes in HBase:
>
>   - Script to add/remove regionservers and datanodes from a cluster when
>   cluster is live
>
>
>   - Script to handle master node failure
>   - Script for graceful shutdown/startup of hbase
>
> Is there any adminsitration tool present for hbase. How can we use Hadoop
> On
> Demand and/or zookeeper to make the cluster failure resistant.
> Regards,
> Ninad
>


Re: Administration tool for HBase

2009-05-14 Thread Ryan Rawson
Dont kill database servers with kill -9 :-)

hbase-daemon.sh stop regionserver does a graceful shutdown, which can take a
while as data is flushed.

there was a bug, which i indicated, which can prevent root/meta from being
reassigned properly.

a few other notes:
- adding servers is easy, just start the new regionservers, you are set
- removing is tricky since you have to wait for the flush to happen, and the
bug in 1362.

good luck!

On Thu, May 14, 2009 at 3:03 AM, Ninad Raut wrote:

> Ryan,
> Using bin/hbase-daemon.sh start regionserver and bin/hbase-daemon.sh stop
> regionserver can we add/remove slave nodes when the cluster is live?
> How to handle a region not serving exception? Because in this scenario the
> daemon.sh script seems to go on forever?
> We usually kill the HRegion process using kill -9, which causes loss of
> data
> if it is not flushed.
> Regards,
> Ninad
>
> On Thu, May 14, 2009 at 12:26 PM, Ryan Rawson  wrote:
>
> > There are tools:
> >
> > bin/start-hbase.sh
> > bin/stop-hbase.sh
> >
> > starts/stops cluster.
> >
> > per node:
> > bin/hbase-daemon.sh start regionserver
> > bin/hbase-daemon.sh stop regionserver
> >
> > you might need HBASE-1362 however.
> >
> > On Wed, May 13, 2009 at 11:46 PM, Ninad Raut  > >wrote:
> >
> > > Hi,
> > > I have been asekd to automate the following processes in HBase:
> > >
> > >   - Script to add/remove regionservers and datanodes from a cluster
> when
> > >   cluster is live
> > >
> > >
> > >   - Script to handle master node failure
> > >   - Script for graceful shutdown/startup of hbase
> > >
> > > Is there any adminsitration tool present for hbase. How can we use
> Hadoop
> > > On
> > > Demand and/or zookeeper to make the cluster failure resistant.
> > > Regards,
> > > Ninad
> > >
> >
>


Re: Problems when executing many (?) HTable.lockRow()

2009-05-14 Thread Ryan Rawson
Ah I hate to be a cloud on a sunny day, but iirc, zk isn't designed for a
high write load. With thousands of requests a second one could overwhelm the
zk paxos consensus seeking protocol.

Another thing to remember is hbase doesn't "overwrite" values, it just
versions them. Perhaps this can be of help?

On May 14, 2009 11:41 AM, "stack"  wrote:

No consideration has been made for changes in how locks are done in new
0.20.0 API. Want to propose...

On Thu, May 14, 2009 at 9:44 AM, Guilherme Germoglio
wrote:


> This way, HTable could directly request for read or write row locks ( >
http://hadoop.apache.org/z...


Re: Problems when executing many (?) HTable.lockRow()

2009-05-14 Thread Ryan Rawson
I stand corrected, thanks for the info!

-ryan

On Thu, May 14, 2009 at 1:07 PM, Joey Echeverria  wrote:

> With a 5 server zk ensemble and an 80% write ratio, you should be able
> to support about 10,000 operations per second[1]. That sounds
> reasonable to me for most uses that require locks. If you require
> higher performance than that, then locking probably isn't for you.
> Taking advantage of versioning or other optimistic concurrency control
> is probably necessary.
>
> At a minimum, I think it makes sense to have zk-based locks as an
> alternative to the current locks which tie up an RPC thread. Testing
> will probably required to see how it performs under various
> assumptions.
>
> [1]
> http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperOver.html#Performance
>
> On Thu, May 14, 2009 at 7:52 PM, Ryan Rawson  wrote:
> > Ah I hate to be a cloud on a sunny day, but iirc, zk isn't designed for a
> > high write load. With thousands of requests a second one could overwhelm
> the
> > zk paxos consensus seeking protocol.
> >
> > Another thing to remember is hbase doesn't "overwrite" values, it just
> > versions them. Perhaps this can be of help?
> >
> > On May 14, 2009 11:41 AM, "stack"  wrote:
> >
> > No consideration has been made for changes in how locks are done in new
> > 0.20.0 API. Want to propose...
> >
> > On Thu, May 14, 2009 at 9:44 AM, Guilherme Germoglio
> > wrote:
> >
> >
> >> This way, HTable could directly request for read or write row locks ( >
> > http://hadoop.apache.org/z...
> >
>


Re: Problems when executing many (?) HTable.lockRow()

2009-05-14 Thread Ryan Rawson
Given the non core nature, I think the api should potentially facilitate
this but the code should be contrib.

On May 14, 2009 5:32 PM, "Guilherme Germoglio"  wrote:

On Thu, May 14, 2009 at 3:40 PM, stack  wrote: > No
consideration has been made f...
I think so.

If nothing is to be changed on RowLock class, we could use the following
approach:

Considering the line as is today:

*RowLock lock = htable.lockRow(row);*

*htable.lockRow(row)*, instead of contacting the regionserver and requesting
a lock for the given row, it would contact zk for a lock, just as the lock
recipe<
http://hadoop.apache.org/zookeeper/docs/current/recipes.html#sc_recipes_Locks
>[1].
Notice that it will be waiting for the lock just as it does today, the
only difference is that regionserver resources (e.g., RPC thread) aren't
used. After it receives the lock, htable would: (1) randomly generate a
lockid, (2) put an entry in a Map, (3) create a
RowLock using the lockid and (4) return the method.

>From this point, any operation could be performed even without passing
rowlock as parameter, zookeeper + the implementation of the lock recipe in
htable are now ensuring that no other client would be performing any
operation concurrently on the given row. [2]

Finally, *htable.unlockRow(lock)* must be invoked, which would make Htable
delete the matching zk node (remember the Map).

One good thing of this approach is that we don't have to worry about lock
leases: if the client dies, zk will notice at some point in the future and
release the lock. And if the client forgets to unlock the row, its code is
wrong. (:

However, if we are to redesign the API, I would propose write and read
locks. Then htable would have two methods: HTable.lockRowForRead(row),
HTable.lockRowForWrite(row) [3] and the lock recipe to be implemented would
be the Shared Locks
recipe<
http://hadoop.apache.org/zookeeper/docs/current/recipes.html#Shared+Locks>.


[1] We may design carefully how the locknode would be created according to
zk constraints on how many nodes it can manage in a single directory. Maybe
we should do something like:
/hbase/locks/table-name/hash-function(row)/row/{read-, write-}

[2] I agree that we are not protecting ourselves from a malicious client
using HTable, who could simply "forget" to request the lock for the given
row and then mess everything. But this is how it's everywhere, isn't it?

[3] Suggest better method names, please!

> St.Ack > > > On Thu, May 14, 2009 at 9:44 AM, Guilherme Germoglio <
germog...@gmail.com > >wrote:...
--

Guilherme msn: guigermog...@hotmail.com homepage:
http://germoglio.googlepages.com


Re: Administration tool for HBase

2009-05-15 Thread Ryan Rawson
Please consider enabling debug logs:

http://wiki.apache.org/hadoop/Hbase/FAQ#5

Also if you are having a problem with a process going down, please do 'kill
-QUIT " to get a stack dump into the .out file.  This helps us diagnose
your problem.

Until http://issues.apache.org/jira/browse/HADOOP-4379 is fixed, kill -9 of
a regionserver will lose data.  We are putting in extra logfile closes in
0.20, but you stand to risk up to ~60mbish of edits if you are under a heavy
write load.

People _are_ working on HADOOP-4379 as we speak.

On Fri, May 15, 2009 at 12:39 AM, Sasha Dolgy  wrote:

> you need to find the root cause.  i've had this before and it worked out to
> be a permissions issue on some of the files.  log files are your friend...
>
> On Fri, May 15, 2009 at 8:14 AM, Ninad Raut  >wrote:
>
> > Hi Andy,
> > But what if the daemon.sh keep running for ever? It just doent complete
> > even
> > after an hour or so. Any other workaround?
> >
> > On Thu, May 14, 2009 at 10:44 PM, Andrew Purtell  > >wrote:
> >
> > > NEVER KILL -9 A REGION SERVER!!!
> > >
> > >   - Andy
> > >
> > >
> > >
> > >
> > > 
> > > From: Ninad Raut 
> > > To: hbase-user@hadoop.apache.org
> > > Cc: Ranjit Nair 
> > > Sent: Thursday, May 14, 2009 3:03:06 AM
> > > Subject: Re: Administration tool for HBase
> > >
> > > Ryan,
> > > Using bin/hbase-daemon.sh start regionserver and bin/hbase-daemon.sh
> stop
> > > regionserver can we add/remove slave nodes when the cluster is live?
> > > How to handle a region not serving exception? Because in this scenario
> > the
> > > daemon.sh script seems to go on forever?
> > > We usually kill the HRegion process using kill -9, which causes loss of
> > > data
> > > if it is not flushed.
> > > Regards,
> > > Ninad
> > >
> > > On Thu, May 14, 2009 at 12:26 PM, Ryan Rawson 
> > wrote:
> > >
> > > > There are tools:
> > > >
> > > > bin/start-hbase.sh
> > > > bin/stop-hbase.sh
> > > >
> > > > starts/stops cluster.
> > > >
> > > > per node:
> > > > bin/hbase-daemon.sh start regionserver
> > > > bin/hbase-daemon.sh stop regionserver
> > > >
> > > > you might need HBASE-1362 however.
> > > >
> > > > On Wed, May 13, 2009 at 11:46 PM, Ninad Raut <
> > hbase.user.ni...@gmail.com
> > > > >wrote:
> > > >
> > > > > Hi,
> > > > > I have been asekd to automate the following processes in HBase:
> > > > >
> > > > >   - Script to add/remove regionservers and datanodes from a cluster
> > > when
> > > > >   cluster is live
> > > > >
> > > > >
> > > > >   - Script to handle master node failure
> > > > >   - Script for graceful shutdown/startup of hbase
> > > > >
> > > > > Is there any adminsitration tool present for hbase. How can we use
> > > Hadoop
> > > > > On
> > > > > Demand and/or zookeeper to make the cluster failure resistant.
> > > > > Regards,
> > > > > Ninad
> > > > >
> > > >
> > >
> > >
> > >
> > >
> > >
> >
>
>
>
> --
> Sasha Dolgy
> sasha.do...@gmail.com
>


Re: Loading large resultset into HBase

2009-05-16 Thread Ryan Rawson
A batch update holds the puts for a single row, are you really updating 5k
cols per row?

generally i turn autoflushing off, use a 1-12mb client buffer.

even so, you can see 'cant find address' or 'not serving region exception'
during a transition of regions.  check the region historian for the region
in question to see if there were moves.

one other thing, are you getting exceptions throw from table.commit() ?  the
existing client will retry to smooth over region transitions, so you can get
exceptions in the logfile but not actually be thrown from table.commit().

One thing I do is do my uploads using java programs and map reduces - this
ensures maximal performance and lets me do automated retries.  24m rows
should import in no time.  I can do about 500m rows in 4 hours...

good luck!
-ryan

On Fri, May 15, 2009 at 4:53 PM, llpind  wrote:

>
> Hey All,
>
> Finally finished.  But didn't complete (~ between 24,770,000 - 25,760,000
> records processed).  Got the following exception before java upload app
> died:
>
> org.apache.hadoop.hbase.client.NoServerForRegionException: No server
> address
> listed in .META. for region
>
> I believe it has to do with "Problem: DFS instability and/or regionserver
> lease timeouts" (http://wiki.apache.org/hadoop/Hbase/Troubleshooting).  So
> i
> was going to try to put some JVM flags in for GC.  Also, when creating the
> table, is there any column options I should use that will help in upload?
>
> When i try to get results from within HBase shell it gives the following
> exception:
>
> org.apache.hadoop.hbase.NotServingRegionException:
> org.apache.hadoop.hbase.NotServingRegionException:
> ,,1242429308026
>at
>
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(HRegionServer.java:2071)
>at
>
> org.apache.hadoop.hbase.regionserver.HRegionServer.getRow(HRegionServer.java:1534)
>at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>at
>
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>at
>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>at java.lang.reflect.Method.invoke(Method.java:597)
>at
> org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:632)
>at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:912)
>
>
>from org/apache/hadoop/hbase/client/HConnectionManager.java:858:in
> `getRegionServerWithRetries'
>from org/apache/hadoop/hbase/client/HTable.java:608:in `getRow'
>from org/apache/hadoop/hbase/client/HTable.java:583:in `getRow'
>from org/apache/hadoop/hbase/client/HTable.java:478:in `getRow'
>from org/apache/hadoop/hbase/client/HTable.java:425:in `getRow'
>from sun/reflect/NativeMethodAccessorImpl.java:-2:in `invoke0'
>from sun/reflect/NativeMethodAccessorImpl.java:39:in `invoke'
>from sun/reflect/DelegatingMethodAccessorImpl.java:25:in `invoke'
>from java/lang/reflect/Method.java:597:in `invoke'
>from org/jruby/javasupport/JavaMethod.java:298:in
> `invokeWithExceptionHandling'
>from org/jruby/javasupport/JavaMethod.java:259:in `invoke'
>from org/jruby/java/invokers/InstanceMethodInvoker.java:44:in `call'
>from org/jruby/runtime/callsite/CachingCallSite.java:280:in
> `cacheAndCall'
>from org/jruby/runtime/callsite/CachingCallSite.java:116:in `call'
>from org/jruby/ast/CallOneArgNode.java:57:in `interpret'
>from org/jruby/ast/LocalAsgnNode.java:120:in `interpret'
> ... 120 levels...
>from org/jruby/internal/runtime/methods/DynamicMethod.java:226:in
> `call'
>from org/jruby/internal/runtime/methods/CompiledMethod.java:216:in
> `call'
>from org/jruby/internal/runtime/methods/CompiledMethod.java:71:in
> `call'
>from org/jruby/runtime/callsite/CachingCallSite.java:260:in
> `cacheAndCall'
>from org/jruby/runtime/callsite/CachingCallSite.java:75:in `call'
>from home/hadoop/hbase19/bin/$_dot_dot_/bin/hirb.rb:441:in
> `__file__'
>from home/hadoop/hbase19/bin/$_dot_dot_/bin/hirb.rb:-1:in `__file__'
>from home/hadoop/hbase19/bin/$_dot_dot_/bin/hirb.rb:-1:in `load'
>from org/jruby/Ruby.java:564:in `runScript'
>from org/jruby/Ruby.java:467:in `runNormally'
>from org/jruby/Ruby.java:340:in `runFromMain'
>from org/jruby/Main.java:214:in `run'
>from org/jruby/Main.java:100:in `run'
>from org/jruby/Main.java:84:in `main'
>from /home/hadoop/hbase19/bin/../bin/hirb.rb:338:in `get'
>
>
>
> I am open to any other ideas.  Should my batch update be smaller than
> 10,000? ...5,000?  I'd like to make any adjustments before i start this
> long
> upload again.  Thanks.
> --
> View this message in context:
> http://www.nabble.com/Loading-large-resultset-into-HBase-tp23566568p23568944.html
> Sent from the HBase User mailing list archive at Nabble.com.
>
>


Re: Execute HQL queries programtically from java

2009-05-20 Thread Ryan Rawson
Hey,

I think you might be confused... there is no "hql". To query you use the
HTable api calls to access your data. Look at get and openscanner.

On May 20, 2009 3:26 AM, "monty123"  wrote:


Hi All,

I am able to play with tables ( add, update, insert etc. ) from a java class
using HTable, BatchUpdate apis.

Is there are any apis to directly execute hql queries from a java program,
just like in jdbc where we can execute Sql queries programtically.
for eg:- execute put, get etc. hql queries programtically

Please help.
Thanks in Advance.

--
View this message in context:
http://www.nabble.com/Execute-HQL-queries-programtically-from-java-tp23632799p23632799.html
Sent from the HBase User mailing list archive at Nabble.com.


Re: Hbase

2009-05-22 Thread Ryan Rawson
Hey Burcu,

The list archive has a number of queries of the "how do I model data" vein.
There have been many answers as well. Jon Gray has been known to do a brain
dump from time to time.

One of the biggest differences between something like HBase and a SQL
database is the 'no-join factor'.  Since data for different tables are
stored on different regionservers, there is no way HBase to do the join for
you.  There are 2 general ways of doing the join:

- Map-Reduce job, read both tables, join them in the MR and filter out the
unwanted rows.  Obviously not an online application.
- Read one side of the join, then issue N queries to retrieve the other side
of the join.  You may or may not be able to take advantage of scan ranges
for the other side of the join.

Thus you don't want to design a normal relational schema, you'd end up
making too many RPCs.  There are several solutions:
- store more data per row.  With near-unlimited columns, you won't need to
have a different table to have a variable number of columns.
- store serialized data structures in a cell value.  This can help if you
have a set of data that is commonly read at once (like the fields of an
address box).  Thrift and protobuf are 2 well known libraries that do this
for you.
- Duplicate and denormalize data.

Since the main approach by which you can get performance is row-locality, be
sure to exploit the careful design of your row-keys.  Remember they are
sorted in lexographic order, so prepend '0' to those numbers you want in
numerical order. Columns are also stored in lexographical order within a
row, so if your plans include a very large number of columns, you may want
to exploit that as well.  While hbase 0.19.3 have issues with extremely wide
columns, these problems are being actively addressed in 0.20 and beyond.

-ryan

2009/5/20 Jim Kellerman (POWERSET) 

>  Questions such as these are better asked on the HBase user mailing list.
> See http://hadoop.apache.org/hbase/mailing_lists.html for how to
> subscribe. There is also an IRC channel on irc.freenode.net the channel's
> name is #hbase By asking questions on the list or in IRC, you are not
> dependent on a single person for an answer (for example, I was out of town
> unexpectedly for three weeks with limited email access).
>
>
>
> However to answer your question, HQL was removed quite some time ago and
> replaced with a jruby shell.
>
> I had thought we had removed all references to HQL from the wiki some time
> ago. If you could point us to the page you saw it on and how you found the
> page, we can clean it up.
>
>
>
> ---
> Jim Kellerman, Powerset (Live Search, Microsoft Corporation)
>
>
>
> *From:* Burcu Hamamcioglu [mailto:burcu.hamamcio...@tikle.com]
> *Sent:* Wednesday, May 20, 2009 7:48 AM
> *To:* j...@apache.org; Jim Kellerman (POWERSET)
> *Subject:* Hbase
>
>
>
> Hi Jim,
>
> I have 2 questions about hbase. In one of your documents I've seen that you
> can open the HQL window( just only for selecting data from tables). How can
> I open this web UI? From which port default?
>
>
>
> My second question is about getting know Hbase. I am a newbie on hbase and
> really confused understanding  it's data modelling. Is there any way or book
> that you can suggest me about Hbase data modelling? How can I achieve Hbase
> ?
>
>
>
> Sorry if bothering you with this email.
>
>
>
> Thanks in advance
>
>
>
> *Burcu Hamamcıoğlu* [image:
> http://www.tikle.com/images/signature/z-logo1.png] 
>
>
>
> [image: cid:image003.gif@01C7D8E0.DDE2F120] +90.212.285 1214 / 306
>
> [image: http://www.tikle.com/images/signature/spacer.gif][image:
> http://www.tikle.com/images/signature/fax-z.png] +90.212.285 1217
>
> burcu.hamamcio...@tikle.com
>
>
>


Re: Urgent: HBase Data Lost

2009-05-25 Thread Ryan Rawson
Maybe stack will chime in here with a potential recovery mechanism, but Erik
is correct.  What has happened is the metadata which indicates which tables
exist, and what their ranges are has disappeared.  Right now there is no
easy way to recover back to the original because of the missing metadata is
not stored elsewhere. What you can try in the mean time is directly
accessing the mapfiles using the raw mapfile reader (i think the class is
Mapfile...) - you'd be able to get the data out, then you could re-insert it
back into a running instance later.

Needless to say, one should not trust a 1-node cluster to irreplacable
data.   Until certain HDFS bugs are resolved, which are slated for a hbase
0.20 timeline, there is always a data loss hole.  Good news is there might
be a backported HDFS 0.19 patch, but that may not be relevant since HBase
0.20 is based on Hadoop/HDFS 0.20.

good luck...

-ryan

On Mon, May 25, 2009 at 12:24 PM, Erik Holstad wrote:

> Hey Arber!
> What it sounds like to me is that the table Meta hadn't been flushed to
> disk
> and was inly sitting on memory, so
> when the machine went down that data got lost.
>
> Regards Erik
>


Re: Urgent: HBase Data Lost

2009-05-25 Thread Ryan Rawson
Hopefully this week we can provide advisory of a set of patches that
ameloriates this problem. Stay tuned!

On May 25, 2009 8:07 PM, "Yabo-Arber Xu"  wrote:

Hi Erik and Ryon,

Thanks for your reply. This again proved how important  fault-torrent is. It
seems that I have to code a bit to see if I can extract the data out.

Best,
Arber

On Tue, May 26, 2009 at 3:32 AM, Ryan Rawson  wrote: >
Maybe stack will chime ...


Re: Obtaining the Timestamp of my last insert/update

2009-05-25 Thread Ryan Rawson
For this case, I am with Erik, use an explicit timestamp, what does it
matter if the timestamp is set on the client or server side, milliseconds
apart?

A more philosophical question for the other devs, should we consider
depreciating the use of the explicit row lock... lock acquisition is slow
and dangerous, since it can DOS the server.  Or perhaps change the semantics
to have a low-lock-wait period (lets say 20-50ms?), so the client can
busy-wait the server instead of tying up IPC threads?





On Mon, May 25, 2009 at 10:04 PM, Erik Holstad wrote:

> Hi Bob!
> What would be the use case where you could use an explicit timestamp
> instead
> of using the rowlock?
> To me they are used for different things, so what are you planning to to
> with them?
>
> Regards Erik
>


Filter use cases

2009-05-26 Thread Ryan Rawson
Hi all,

With HBASE-1304, it's time to normalize and review our filter API.

Here are a few givens:
- all calls must be byte[] offset,int offset, int length
- maybe we can have calls for KeyValue (which encodes all parts of the key &
value as per the name)
- we'd like to get rid of the calls:
--   boolean filterRow(final SortedMap columns);
--   boolean filterRow(final List results);
These calls are expensive, and there is no reason to have them.

Here is a proposal, imagine a filter will see this sequence of calls:
- reset()
- filterRowKey(byte[],int,int) - true to include row, false to skip to next
row
- filterKeyValue(KeyValue) - true to include key/value, false to skip
-- can choose to filter on family, qualifier, value, anything really.
- filterRow() - true to include entire row, false to post-hoc veto row

In this case one could implement the "filterIfColumnMissing" feature of
ColumnValueFilter by carrying state and returning false from filterRow() to
veto the row based on the columns/values we didn't see.

In any of these cases, all these functions will be called quite frequently,
so efficiency of the code is paramount.  It's probable that filterRowKey()
will be 'cached' by the calling code, but filterKeyValue() is called for
nearly every single value we would normally return (ie: it's applied _AFTER_
column matching and version and timestamp and delete tracking).

The goal is to:
(a) make the implementation easy and performant
(b) make the API normative and easy to code for
(c) make everything work

Thoughts?
-ryan


Re: htable.getScanner() slow?

2009-05-28 Thread Ryan Rawson
Hi,

You should consider setting scanner caching to reduce the number of
server-round trips.

But slow scanners is a known problem with 0.19.  HBase 0.20 aims to fix this
substantially.  Shocking speed gains are hopefully going to be par for the
course.

-ryan

On Thu, May 28, 2009 at 6:47 PM, Xinan Wu  wrote:

> Hi,
>
> I've been experimenting row scanning in hbase recently, following
> advice from
> http://devblog.streamy.com/2009/04/23/hbase-row-key-design-for-paging-limit-offset-queries/
> ?.
>
> One thing I notice is htable.getScanner() function call is very slow...
>
> My table schema is very simple. Integer (as binary 4 bytes) as rowKey,
> and single column family..
>
> If I store 100 records in the same row with different columns, I can
> get all the them with a single API call, at about 350 requests per
> second (but paging would not be very scalable if records# gets
> larger).
>
> If I store 100 records in 100 different rows (with sort-key appended
> to rowKey), then I can use scanner to get them (and paging would be
> more scalable). However, getScanner() call takes about 60 ms to
> return, and subsequent scanner.next() calls are very fast. Overall,
> this gives me only 15 requests per second.
>
> My dev box is ubuntu 8.04 2.4GHz Quad, 4GB mem, pretty typical one.
>
> Anyone has experience with slow scanner creation? Any suggestions?
>
> Xinan
>


Re: htable.getScanner() slow?

2009-05-28 Thread Ryan Rawson
The speed gains will be shocking.  Right now you can expect a 5-100x speed
increase, and soon it will be more like 10-20-200x.

I found with 0.19 there was a 200ms floor in my tests, and 0.20 so far has
blown past that.  There is HBASE-1304 still in progress which is showing
much promise.  Please stay tuned!

These are very exciting times for hbase... Soon HBase will be no SPOF apart
from HDFS, and performant as well.

If you are feeling brave, try hadoop 0.20 and hbase-trunk.  Standard
developer-preview type caveats apply, support is semi-limited since the bug
you might have is already being rewritten.

Having said that, I use HBase 0.20-trunk in production.  I'm also a
committer, so YMMV.

Good luck!
-ryan

On Thu, May 28, 2009 at 7:06 PM, Xinan Wu  wrote:

> Ryan,
>
> Thanks for the reply. I tried tweaking scanner caching but did not
> change the speed much. The test I ended up doing was just getScanner()
> and then immedietely scanner.close() without issuing scanner.next()...
>
> Anyway, it's good to know HBase 0.20 may improve the speed. Is slow
> scanner a known issue with hbase < 0.19 too? (I am using 0.19.2/3, but
> am just curious...)
>
> Xinan
>
> On Thu, May 28, 2009 at 6:56 PM, Ryan Rawson  wrote:
> > Hi,
> >
> > You should consider setting scanner caching to reduce the number of
> > server-round trips.
> >
> > But slow scanners is a known problem with 0.19.  HBase 0.20 aims to fix
> this
> > substantially.  Shocking speed gains are hopefully going to be par for
> the
> > course.
> >
> > -ryan
> >
> > On Thu, May 28, 2009 at 6:47 PM, Xinan Wu  wrote:
> >
> >> Hi,
> >>
> >> I've been experimenting row scanning in hbase recently, following
> >> advice from
> >>
> http://devblog.streamy.com/2009/04/23/hbase-row-key-design-for-paging-limit-offset-queries/
> >> ?.
> >>
> >> One thing I notice is htable.getScanner() function call is very slow...
> >>
> >> My table schema is very simple. Integer (as binary 4 bytes) as rowKey,
> >> and single column family..
> >>
> >> If I store 100 records in the same row with different columns, I can
> >> get all the them with a single API call, at about 350 requests per
> >> second (but paging would not be very scalable if records# gets
> >> larger).
> >>
> >> If I store 100 records in 100 different rows (with sort-key appended
> >> to rowKey), then I can use scanner to get them (and paging would be
> >> more scalable). However, getScanner() call takes about 60 ms to
> >> return, and subsequent scanner.next() calls are very fast. Overall,
> >> this gives me only 15 requests per second.
> >>
> >> My dev box is ubuntu 8.04 2.4GHz Quad, 4GB mem, pretty typical one.
> >>
> >> Anyone has experience with slow scanner creation? Any suggestions?
> >>
> >> Xinan
> >>
> >
>


Re: htable.getScanner() slow?

2009-05-28 Thread Ryan Rawson
The version is 0.20 - there was a "0.2.0" which is a really old release.

I've gotten over a million rows/read/sec on a 20 machine cluster.

-ryan

On Thu, May 28, 2009 at 7:22 PM, Ryan J. McDonough wrote:

> That's actually really good to know. I just had my bubble burst today when
> I found that my test results were so bad due slow scanner reads. Now I have
> to pull 0.2.0 to see how much faster it is ;)
>
> Ryan-
>
>
> On May 28, 2009, at 10:11 PM, Ryan Rawson wrote:
>
>  The speed gains will be shocking.  Right now you can expect a 5-100x speed
>> increase, and soon it will be more like 10-20-200x.
>>
>> I found with 0.19 there was a 200ms floor in my tests, and 0.20 so far has
>> blown past that.  There is HBASE-1304 still in progress which is showing
>> much promise.  Please stay tuned!
>>
>> These are very exciting times for hbase... Soon HBase will be no SPOF
>> apart
>> from HDFS, and performant as well.
>>
>> If you are feeling brave, try hadoop 0.20 and hbase-trunk.  Standard
>> developer-preview type caveats apply, support is semi-limited since the
>> bug
>> you might have is already being rewritten.
>>
>> Having said that, I use HBase 0.20-trunk in production.  I'm also a
>> committer, so YMMV.
>>
>> Good luck!
>> -ryan
>>
>> On Thu, May 28, 2009 at 7:06 PM, Xinan Wu  wrote:
>>
>>  Ryan,
>>>
>>> Thanks for the reply. I tried tweaking scanner caching but did not
>>> change the speed much. The test I ended up doing was just getScanner()
>>> and then immedietely scanner.close() without issuing scanner.next()...
>>>
>>> Anyway, it's good to know HBase 0.20 may improve the speed. Is slow
>>> scanner a known issue with hbase < 0.19 too? (I am using 0.19.2/3, but
>>> am just curious...)
>>>
>>> Xinan
>>>
>>> On Thu, May 28, 2009 at 6:56 PM, Ryan Rawson  wrote:
>>>
>>>> Hi,
>>>>
>>>> You should consider setting scanner caching to reduce the number of
>>>> server-round trips.
>>>>
>>>> But slow scanners is a known problem with 0.19.  HBase 0.20 aims to fix
>>>>
>>> this
>>>
>>>> substantially.  Shocking speed gains are hopefully going to be par for
>>>>
>>> the
>>>
>>>> course.
>>>>
>>>> -ryan
>>>>
>>>> On Thu, May 28, 2009 at 6:47 PM, Xinan Wu  wrote:
>>>>
>>>>  Hi,
>>>>>
>>>>> I've been experimenting row scanning in hbase recently, following
>>>>> advice from
>>>>>
>>>>>
>>> http://devblog.streamy.com/2009/04/23/hbase-row-key-design-for-paging-limit-offset-queries/
>>>
>>>> ?.
>>>>>
>>>>> One thing I notice is htable.getScanner() function call is very slow...
>>>>>
>>>>> My table schema is very simple. Integer (as binary 4 bytes) as rowKey,
>>>>> and single column family..
>>>>>
>>>>> If I store 100 records in the same row with different columns, I can
>>>>> get all the them with a single API call, at about 350 requests per
>>>>> second (but paging would not be very scalable if records# gets
>>>>> larger).
>>>>>
>>>>> If I store 100 records in 100 different rows (with sort-key appended
>>>>> to rowKey), then I can use scanner to get them (and paging would be
>>>>> more scalable). However, getScanner() call takes about 60 ms to
>>>>> return, and subsequent scanner.next() calls are very fast. Overall,
>>>>> this gives me only 15 requests per second.
>>>>>
>>>>> My dev box is ubuntu 8.04 2.4GHz Quad, 4GB mem, pretty typical one.
>>>>>
>>>>> Anyone has experience with slow scanner creation? Any suggestions?
>>>>>
>>>>> Xinan
>>>>>
>>>>>
>>>>
>>>
>


Thrift server returning non-existent data

2009-05-30 Thread Ryan Rawson
Hi all,

I had a situation where a thrift gateway was returning data that didnt exist
in the hbase table!  It was very confusing because none of the Java scanners
or the jruby shell would show the data, but the 'getRow' thrift call - which
was tunneled via autossh btw - kept on returning reasonable looking, yet
non-existent data.

I had restarted the regionserver hosting the table (small table), and
twiddled the code over and over.  Finally I restarted the thrift server and
that fixed it.

My only remaining theory after inspecting our thrift server code is that the
thrift RPC/server code will cache replies...

Or some other bug perhaps?

anyone have any thoughts?


Re: trunk code, RetriesExhaustedException

2009-05-30 Thread Ryan Rawson
I'm thinking xciever limit...

Try 6+ machines...

On May 30, 2009 10:37 PM, "Xinan Wu"  wrote:

yes epoll and ulimit -n limits are both upped as suggested by docs. My
"test cluster" is pretty typical local dev box, ubuntu 8.10 64bit, sun
1.6, 4G mem, hdfs namenode/datanode hbase master/regionserver all
running on the same box. My htable has about 10m rows imported when i
hit this state... info pages/logs show all regions are ready but I
can't pull any data out from them.

On Sat, May 30, 2009 at 7:48 AM, stack  wrote: > Tell us
more about your cluster ...


barcampnyc & state of hbase

2009-05-31 Thread Ryan Rawson
Someone gave a talk on HBase at barcamp NYC, here's a tweet:

http://twitter.com/ggodik/statuses/1974136991

We should see what kind of support we can give to this and other random
speakers.

With the publishing of 1304 and 0.20-rc (we'll get there soon!) there will
at the final word in hbase performance (for a little while).

in the mean time, looking forward to:
- HDFS fixes for block not found
- HDFS sync fixes (in 0.21)


Re: State of HA

2009-06-01 Thread Ryan Rawson
Hey,

Stack is saying that for HADOOP-4379, it fails 1/5th of the time - recovery
takes more than 15 minutes, aka potentially unlimited amount of time.  That
patch relies on lease recovery it seems, so it may not be the final answer
for us.

Now, on the subject of the rest of things, under Zookeeper we are doing a
much better job at HA.  Regionserver crashes are detect significantly faster
than the 2 minute lease timeout, with my fixes you can take down any
regionserver without getting 'stuck' with an unassigned ROOT/META
(previously a problem).

I have noticed on trunk I can kill and restart the master w/o taking down
the cluster.  During master start-up it does a fairly good job at detecting
node status and otherwise recovering.  I can't say about master elections
exactly yet.

The HA story is shaping up nicely.

To end on a sour note, HDFS Namenode is still a SPOF.  When we're done with
HBase 0.20 it should be the only SPOF.

-ryan

On Mon, Jun 1, 2009 at 1:50 PM,  wrote:

> I am trying to parse this: are you implying that I can expect a 20% ("1 out
> of 5 or so") success getting HA to work with this code?
>
> -Original Message-
> From: saint@gmail.com [mailto:saint@gmail.com] On Behalf Of ext
> stack
> Sent: 01 June, 2009 13:27
> To: hbase-user@hadoop.apache.org
> Subject: Re: State of HA
>
> You can pull TRUNK and try it with HADOOP-4379.
>
> The master failover works as J-D suggests.  It needs some polish but thats
> on its way.  The HADOOP-4379 will get you a sync that works most of the
> time
> (1 out of 5 or so in my testing) but hopefully that'll be addressed soon
> too.  You'll also need HBASE-1470.   Its the bit of code that exploits
> HADOOP-4379 when configuration is set right).
>
> If you need help setting up stuff, you know where to find us.  Issues we
> want to hear about because we're hoping to tell the above as part of our
> 0.20.0 release story.
>
> Yours,
> St.Ack
>
> On Mon, Jun 1, 2009 at 7:59 AM,  wrote:
>
> > Hello,
> >
> > I have been looking at Jira and trying to get a current snapshot of the
> > state of HA for HBase/Hadoop? I know that the zookeeper integration is
> the
> > core of the HA story, but when is that slated for a "stable" debut? Is
> there
> > anything that is currently in svn that we can pull and test?
> >
> > TIA,
> >
> > Andrew
> >
> >
>


Re: master performance

2009-06-01 Thread Ryan Rawson
Hey,

The issue actually isn't with Master performance, but the performance of
scanning .META. which is done for every single UI load.  I'm not sure there
are any good answers for 0.19.x...

However, once it becomes available, HBase 0.20.0 will answer some of these
for you.  A combination of improved scanners, block caching, LZO-based
compression and other fixes should make HBase much more snappy.

This is obviously not the best answer, but there is a roadmap out of the
slowness.

Good luck!
-ryan

On Mon, Jun 1, 2009 at 5:31 PM, Jeremy Pinkham  wrote:

>
> sorry for the novel...
>
> I've been experiencing some problems with my hbase cluster and hoping
> someone can point me in the right direction.  I have a 40 node cluster
> running 0.19.0.  Each node has 4 cores, 8GB (2GB dedicated to the
> regionserver), and 1TB data disk.  The master is on a dedicated machine
> separate from the namenode and the jobtracker.  There is a single table with
> 4 column families and 3700 regions evenly spread across the 40 nodes.  The
> TTL's match our loading pace well enough that we don't typically see too
> many splits anymore.
>
> In trying to troubleshoot some larger issues with bulk loads on this
> cluster I have created a test scenario to try and narrow the problem based
> on various symptoms.  This test is map/reduce job that is using the
> HRegionPartitioner (as an easy way to generate some traffic to the master
> for meta data).  I've been running this job with various size inputs to
> gauge the effect of different numbers of mappers and have found that as the
> number of concurrent mappers creeps up to what I think are still small
> numbers (<50 mappers), the performance of the master is dramatically
> impacted.  I'm judging the performance here simply by checking the response
> time of the UI on the master, since that has historically been a good
> indication of when the cluster is getting into trouble during our loads
> (which I'm sure could mean a lot of things), although i suppose it's
> possible to two are unrelated.
>
> The UI normally takes about 5-7 seconds to refresh master.jsp.  Running a
> job with 5 mappers doesn't seem to impact it too much, but a job with 38
> mappers makes the UI completely unresponsive for anywhere from 30 seconds to
> several minutes during the run.  During this time, there is nothing
> happening in the logs, scans/gets from within the shell continue to work
> fine, and ganglia/top show the box to be virtually idle.  All links off of
> master.jsp work fine, so I presume it's something about the master pulling
> info from the individual nodes, but those UI's are also perfectly
> responsive.
>
> This same cluster used to run on just 20 nodes without issue, so I'm
> curious if I've crossed some threshold of horizontal scalability or if there
> is just a tuning parameter that I'm missing that might take care of this, or
> if there is something known between 0.19.0 and 0.19.3 that might be a
> factor.
>
> Thanks
>
> jeremy
>
>
> The information transmitted in this email is intended only for the
> person(s) or entity to which it is addressed and may contain confidential
> and/or privileged material. Any review, retransmission, dissemination or
> other use of, or taking of any action in reliance upon, this information by
> persons or entities other than the intended recipient is prohibited. If you
> received this email in error, please contact the sender and permanently
> delete the email from any computer.
>
>


Re: Uses cases for checkAndSave?

2009-06-02 Thread Ryan Rawson
The way I think about checkAndSave might work like this:

Takes a Get() object to specify which row and column to affect
Takes a Result object to verify said data.  This should match the Get()
Takes a Put or maybe Delete to apply if the previous two worked.

-ryan

On Tue, Jun 2, 2009 at 4:51 PM, Guilherme Germoglio wrote:

> Hello!
>
> On Tue, Jun 2, 2009 at 3:58 PM, Erik Holstad 
> wrote:
>
> > Hi!
> >
> > On Tue, Jun 2, 2009 at 11:17 AM, Guilherme Germoglio <
> germog...@gmail.com
> > >wrote:
> >
> > > Hi Erik,
> > >
> > > For now, I'm using checkAndSave in order to make sure that a row is
> only
> > > created but not overwritten by multiple threads. So, checkAndSave is
> > mostly
> > > invoked with a new structure created on the client. Actually, I'm
> > checking
> > > if a specific "deleted" column in empty. If the "deleted" column is not
> > > empty, then the row creation cannot be performed. There are another few
> > > tricky cases I'm using it, but I'm sure that making that Result object
> > more
> > > difficult to create than putting values on a map would be bad for me.
> :-)
> >
> > So you have a row with family and qualifier that you check to see if it
> is
> > empty
> > and if it is you insert a new row? So basically you use it as an atomic
> > rowExist
> > checker or? Are you usually batching this checks or would it be ok with
> > something like:
> >
> > public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
> > byte[] value, Put put){}
> > or
> > public boolean checkAndPut(KeyValue checkKv, Put put){}
> > for now?
> >
>
> Yes. It is ok for me to use the methods above for now.
>
> Just in case you are curious on how I'll be using them, there are two cases
> where I'm using checkAndSave:
>
> The first is like the atomic rowExist checker and it represents 90% of the
> use of checkAndSave. Exactly as you said, I've got a column
> attributes:deleted for every row. When creating a new row, the creation
> only
> happens if this column is empty. When the row creation happens, it is
> assigned a 'false' value to this column. When this column receives a 'true'
> value, that is, the row is to be deleted, the 'hard' removal (a HTable's
> Delete) of the row will be performed asynchronously. Until the 'hard'
> removal happens, a software layer that uses HTable will prevent the use of
> any 'soft' deleted row by checking the attributes:deleted column.
>
> The second case of using checkAndSave is to trigger some actions when a
> specific column is updated. So, I don't check for emptiness, but if a
> previous value continues the same when I'm updating the row. For example,
> let's say I have a users table where I will serialize a User object and put
> it into a row. Among other things, the User object contains an e-mail
> attribute and its change must trigger verification actions, changes on
> other
> tables, whatever. I realized that performing a get for every User update
> just to check whether their e-mail changed or not might not be the better
> approach, since changing e-mail is not a very common operation. So, I
> thought it is better to checkAndSave an user expecting their current e-mail
> value will be the same the one already in the table since this will occur
> many many times more than the opposite. However, if it is the case that the
> current e-mail value is different from the one in the table, triggers are
> fired and then a new update is performed.
>
>
>
> >
> > >
> > > However, here's an idea. What if Put and Delete objects have a field
> > > "condition" (maybe, "onlyIf" would be a better name) which is exactly
> the
> > > map with columns and expected values. So, a given Put or Delete of an
> > > updates list will only happen if those expected values match.
> > >
> >
> > Puts and deletes are pretty much just List which is basically a
> > List.
> > I don't think that we want to add complexity for puts and deletes now
> that
> > we have worked
> > so hard to make it faster and more bare bone.
> >
>
> no problem. (sorry!)
>
>
> >
> >
> > > Also, maybe it should be possible to indicate common expected values
> for
> > > all
> > > updates of a list too, so a client won't have to put in all updates the
> > > same
> > > values if needed. But we must remember to solve the conflicts of
> expected
> > > values.
> > >
> > Not really sure if you mean that we would check the value of a key before
> > inserting the new
> > value? That would mean that you would have to do a get for every
> put/delete
> > which is not
> > something we want in the general case.
> >
> >
> > >
> > > (By the way, I haven't seen the guts of new Puts and Deletes, so I
> don't
> > > know how difficult would it be to implement it -- but I can help, if
> > > necessary)
> > >
> > > Thanks,
> > >
> > > On Tue, Jun 2, 2009 at 2:34 PM, Erik Holstad 
> > > wrote:
> > >
> > > > Hi!
> > > > I'm working on putting checkAndSave back into 0.20 and just want to
> > check
> > > > with the people that are using it how they are us

Re: blockcache always on?

2009-06-04 Thread Ryan Rawson
In 0.20 its a global setting right now.  We hope to support in memory column
families soon.

What do you think the appropriate setting should be? My thought is to avoid
block cache on very large blocks and otherwise let the lru do its job.

On Jun 4, 2009 11:25 PM, "Billy Pearson"  wrote:

I created a table with one column in my cluster and started putting data in
to it I noticed that even with blockcache = false it still uses the block
cache
Is this a know problem or is there plans to remove the option from the table
create alter commands?

Billy


Re: Frequent changing rowkey - HBase insert

2009-06-06 Thread Ryan Rawson
Have a look at:

http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html

-ryan


On Sat, Jun 6, 2009 at 4:55 PM, llpind  wrote:

>
> I'm doing an insert operation using the java API.
>
> When inserting data where the rowkey changes often, it seems the inserts go
> really slow.
>
> Is there another method for doing inserts of this type?  (instead of
> BatchUpdate).
>
> Thanks
> --
> View this message in context:
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23906724.html
> Sent from the HBase User mailing list archive at Nabble.com.
>
>


Re: Frequent changing rowkey - HBase insert

2009-06-06 Thread Ryan Rawson
Don't use the thrift gateway for bulk import.

Use the Java API, and be sure to turn off auto flushing and use a reasonably
sizable commit buffer. 1-12MB is probably ideal.

i can push a 20 node cluster past 180k inserts/sec using this.

On Sat, Jun 6, 2009 at 5:51 PM, llpind  wrote:

>
> Thanks Ryan, well done.
>
> I have no experience using Thrift gateway, could you please provide some
> actual code here or in your blog post?  I'd love to see how your method
> compares with mine.
>
> Last night I was able to do ~58 million records in ~1.6 hours using the
> HBase Java API directly.  But with this new data, I'm seeing much slower
> times.  After reading around, it appears it's because my row key now
> changes
> often, whearas before it was constant for some time (more columns).  Thanks
> again. :)
>
>
> Ryan Rawson wrote:
> >
> > Have a look at:
> >
> >
> http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html
> >
> > -ryan
> >
> >
> > On Sat, Jun 6, 2009 at 4:55 PM, llpind  wrote:
> >
> >>
> >> I'm doing an insert operation using the java API.
> >>
> >> When inserting data where the rowkey changes often, it seems the inserts
> >> go
> >> really slow.
> >>
> >> Is there another method for doing inserts of this type?  (instead of
> >> BatchUpdate).
> >>
> >> Thanks
> >> --
> >> View this message in context:
> >>
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23906724.html
> >> Sent from the HBase User mailing list archive at Nabble.com.
> >>
> >>
> >
> >
>
> --
> View this message in context:
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23906943.html
> Sent from the HBase User mailing list archive at Nabble.com.
>
>


Re: Frequent changing rowkey - HBase insert

2009-06-06 Thread Ryan Rawson
In 0.20 things should get faster.

Generally speaking I find HBase's insert performance really good.  One of
the best even.  Plus Just Add Servers (tm).

-ryan

On Sat, Jun 6, 2009 at 6:13 PM, llpind  wrote:

>
> Thanks Ryan,
>
> Yeah that sped it up a bit.
>
> I set :
>table.setAutoFlush(false);
>table.setWriteBufferSize(1024*1024*12);
>
> And it's inserting 1M in about 1 minute+ .   Not the best still.
>
> 2009-06-06 18:06:54.894 ==PROCESSING RECORD: == @100
> 2009-06-06 18:08:07.725 ==PROCESSING RECORD: == @200
> 2009-06-06 18:09:24.992 ==PROCESSING RECORD: == @300
> 2009-06-06 18:11:13.279 ======PROCESSING RECORD: == @400
>
>
> Ryan Rawson wrote:
> >
> > Don't use the thrift gateway for bulk import.
> >
> > Use the Java API, and be sure to turn off auto flushing and use a
> > reasonably
> > sizable commit buffer. 1-12MB is probably ideal.
> >
> > i can push a 20 node cluster past 180k inserts/sec using this.
> >
> > On Sat, Jun 6, 2009 at 5:51 PM, llpind  wrote:
> >
> >>
> >> Thanks Ryan, well done.
> >>
> >> I have no experience using Thrift gateway, could you please provide some
> >> actual code here or in your blog post?  I'd love to see how your method
> >> compares with mine.
> >>
> >> Last night I was able to do ~58 million records in ~1.6 hours using the
> >> HBase Java API directly.  But with this new data, I'm seeing much slower
> >> times.  After reading around, it appears it's because my row key now
> >> changes
> >> often, whearas before it was constant for some time (more columns).
> >> Thanks
> >> again. :)
> >>
> >>
> >> Ryan Rawson wrote:
> >> >
> >> > Have a look at:
> >> >
> >> >
> >>
> http://ryantwopointoh.blogspot.com/2009/01/performance-of-hbase-importing.html
> >> >
> >> > -ryan
> >> >
> >> >
> >> > On Sat, Jun 6, 2009 at 4:55 PM, llpind 
> wrote:
> >> >
> >> >>
> >> >> I'm doing an insert operation using the java API.
> >> >>
> >> >> When inserting data where the rowkey changes often, it seems the
> >> inserts
> >> >> go
> >> >> really slow.
> >> >>
> >> >> Is there another method for doing inserts of this type?  (instead of
> >> >> BatchUpdate).
> >> >>
> >> >> Thanks
> >> >> --
> >> >> View this message in context:
> >> >>
> >>
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23906724.html
> >> >> Sent from the HBase User mailing list archive at Nabble.com.
> >> >>
> >> >>
> >> >
> >> >
> >>
> >> --
> >> View this message in context:
> >>
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23906943.html
> >> Sent from the HBase User mailing list archive at Nabble.com.
> >>
> >>
> >
> >
>
> --
> View this message in context:
> http://www.nabble.com/Frequent-changing-rowkey---HBase-insert-tp23906724p23907040.html
> Sent from the HBase User mailing list archive at Nabble.com.
>
>


Re: scanner on a given column: whole table scan or just the rows that have values

2009-06-09 Thread Ryan Rawson
Without the use of indexes, there is no easy way to get the info without
touching every row.

So yes you'll be scanning every row.  But hbase has good bulk scan perf.

On Jun 9, 2009 7:24 PM, "Ric Wang"  wrote:

How does the scanner know how to get ONLY the "relevant" rows, without a
whole table scan?

Thanks!
Ric

On Tue, Jun 9, 2009 at 4:31 PM, Naveen Koorakula  wrote:
> The scanner only s...
--

Ric Wang wqt.w...@gmail.com


Re: scanner on a given column: whole table scan or just the rows that have values

2009-06-09 Thread Ryan Rawson
Via parallelism? Just add machines. Also a simpler on disk format has
immutable files allows for rapid scanning without concurrency issues during
writes

On Jun 9, 2009 9:44 PM, "Ric Wang"  wrote:

Hi,

Thanks. But if it is still scanning EVERY row in the entire table, how does
HBase achieve better scan performance, compared to a row-based database?

Thanks,
Ric

On Tue, Jun 9, 2009 at 9:35 PM, Ryan Rawson  wrote: >
Without the use of ind...
--

Ric Wang wqt.w...@gmail.com


Re: HBase Failing on Large Loads

2009-06-10 Thread Ryan Rawson
That is a client exception that is a sign of problems on the
regionserver...is it still running? What do the logs look like?

On Jun 10, 2009 2:51 PM, "Bradford Stephens" 
wrote:

OK, I've tried all the optimizations you've suggested (still running
with a M/R job). Still having problems like this:

org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
contact region server 192.168.18.15:60020 for region
joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
'291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
Exceptions:
java.io.IOException: Call to /192.168.18.15:60020 failed on local
exception: java.io.EOFException
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused
java.net.ConnectException: Call to /192.168.18.15:60020 failed on
connection exception: java.net.ConnectException: Connection refused

On Wed, Jun 10, 2009 at 12:40 AM, stack wrote: > On Tue,
Jun 9, 2009 at 11:51 AM,...


Re: HBase Failing on Large Loads

2009-06-10 Thread Ryan Rawson
Hey,

Looks lke you have some HDFS issues.

Things I did to make myself stable:

- run HDFS with -Xmx=2000m
- run HDFS with 2047 xciever limit (goes into hdfs-core.xml or
hadoop-site.xml)
- ulimit -n 32k - also important

With this I find that HDFS is very stable, I've imported hundreds of gigs.

You want to make sure the HDFS xciever limit is set in the hadoop/conf
directory, copied to every node and HDFS restarted.  Also sounds like you
might have a cluster with multiple versions of hadoop.  Double check that!

you're close!
-ryan

On Wed, Jun 10, 2009 at 3:32 PM, Bradford Stephens <
bradfordsteph...@gmail.com> wrote:

> Thanks so much for all the help, everyone... things are still broken,
> but maybe we're getting close.
>
> All the regionservers were dead by the time the job ended.  I see
> quite a few error messages like this:
>
> (I've put the entirety of the regionserver logs on pastebin:)
> http://pastebin.com/m2e6f9283
> http://pastebin.com/mf97bd57
>
> 2009-06-10 14:47:54,994 ERROR
> org.apache.hadoop.hbase.regionserver.HRegionServer: unable to process
> message: MSG_REGION_OPEN:
> joinedcontent,1DCC1616F7C7B53B69B5536F407A64DF,1244667570521:
> safeMode=false
> java.lang.NullPointerException
>
> There's also a scattering of messages like this:
> 2009-06-10 13:49:02,855 WARN
> org.apache.hadoop.hbase.regionserver.HLog: IPC Server handler 1 on
> 60020 took 3267ms appending an edit to HLog; editcount=21570
>
> aaand
>
> 2009-06-10 14:03:27,270 INFO
> org.apache.hadoop.hbase.regionserver.HLog: Closed
>
> hdfs://dttest01:54310/hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667757560,
> entries=16. New log writer:
> /hbase-0.19/log_192.168.18.49_1244659862699_60020/hlog.dat.1244667807249
> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
> Exception in createBlockOutputStream java.io.IOException: Bad connect
> ack with firstBadLink 192.168.18.47:50010
> 2009-06-10 14:03:28,160 INFO org.apache.hadoop.hdfs.DFSClient:
> Abandoning block blk_4831127457964871573_140781
> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
> Exception in createBlockOutputStream java.io.IOException: Could not
> read from stream
> 2009-06-10 14:03:34,170 INFO org.apache.hadoop.hdfs.DFSClient:
> Abandoning block blk_-6169186743102862627_140796
> 2009-06-10 14:03:34,485 INFO
> org.apache.hadoop.hbase.regionserver.MemcacheFlusher: Forced flushing
> of joinedcontent,1F2F64F59088A3B121CFC66F7FCBA2A9,1244667654435
> because global memcache limit of 398.7m exceeded; currently 399.0m and
> flushing till 249.2m
>
> Finally, I saw this when I stopped and re-started my cluster:
>
> 2009-06-10 15:29:09,494 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.18.16:50010,
> storageID=DS-486600617-192.168.18.16-50010-1241838200467,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Version Mismatch
>at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:81)
>at java.lang.Thread.run(Thread.java:619)
>
>
> On Wed, Jun 10, 2009 at 2:55 PM, Ryan Rawson wrote:
> > That is a client exception that is a sign of problems on the
> > regionserver...is it still running? What do the logs look like?
> >
> > On Jun 10, 2009 2:51 PM, "Bradford Stephens"  >
> > wrote:
> >
> > OK, I've tried all the optimizations you've suggested (still running
> > with a M/R job). Still having problems like this:
> >
> > org.apache.hadoop.hbase.client.RetriesExhaustedException: Trying to
> > contact region server 192.168.18.15:60020 for region
> > joinedcontent,242FEB3ED9BE0D8EF3856E9C4251464C,1244666594390, row
> > '291DB5C7440B0A5BDB0C12501308C55B', but failed after 10 attempts.
> > Exceptions:
> > java.io.IOException: Call to /192.168.18.15:60020 failed on local
> > exception: java.io.EOFException
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.ConnectException: Call to /192.168.18.15:60020 failed on
> > connection exception: java.net.ConnectException: Connection refused
> > java.net.Connec

Re: Help with Map/Reduce program

2009-06-10 Thread Ryan Rawson
Hey,

A scanner's lease expires in 60 seconds.  I'm not sure what version you are
using, but try:
table.setScannerCaching(1);

This way you won't retrieve 60 rows that each take 1-2 seconds to process.

This is the new default value in 0.20, but I don't know if it ended up in
0.19.x anywhere.


On Wed, Jun 10, 2009 at 2:14 PM, llpind  wrote:

>
> Okay, I think I got it figured out.
>
> although when scanning large row keys I do get the following exception:
>
> NativeException: java.lang.RuntimeException:
> org.apache.hadoop.hbase.UnknownScannerException:
> org.apache.hadoop.hbase.UnknownScannerException: -4424757523660246367
>at
>
> org.apache.hadoop.hbase.regionserver.HRegionServer.close(HRegionServer.java:1745)
>at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
>at
>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>at java.lang.reflect.Method.invoke(Method.java:597)
>at
> org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:632)
>at
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:912)
>
>from org/apache/hadoop/hbase/client/HTable.java:1741:in `hasNext'
>from sun/reflect/NativeMethodAccessorImpl.java:-2:in `invoke0'
>from sun/reflect/NativeMethodAccessorImpl.java:39:in `invoke'
>from sun/reflect/DelegatingMethodAccessorImpl.java:25:in `invoke'
>from java/lang/reflect/Method.java:597:in `invoke'
>from org/jruby/javasupport/JavaMethod.java:298:in
> `invokeWithExceptionHandling'
>from org/jruby/javasupport/JavaMethod.java:259:in `invoke'
>from org/jruby/java/invokers/InstanceMethodInvoker.java:36:in `call'
>from org/jruby/runtime/callsite/CachingCallSite.java:73:in `call'
>from org/jruby/ast/CallNoArgNode.java:61:in `interpret'
>from org/jruby/ast/WhileNode.java:124:in `interpret'
>from org/jruby/ast/NewlineNode.java:101:in `interpret'
>from org/jruby/ast/BlockNode.java:68:in `interpret'
>from org/jruby/internal/runtime/methods/DefaultMethod.java:156:in
> `interpretedCall'
>from org/jruby/internal/runtime/methods/DefaultMethod.java:133:in
> `call'
>from org/jruby/internal/runtime/methods/DefaultMethod.java:246:in
> `call'
> ... 108 levels...
>from org/jruby/internal/runtime/methods/DynamicMethod.java:226:in
> `call'
>from org/jruby/internal/runtime/methods/CompiledMethod.java:216:in
> `call'
>from org/jruby/internal/runtime/methods/CompiledMethod.java:71:in
> `call'
>from org/jruby/runtime/callsite/CachingCallSite.java:260:in
> `cacheAndCall'
>from org/jruby/runtime/callsite/CachingCallSite.java:75:in `call'
>from home/hadoop/hbase193/bin/$_dot_dot_/bin/hirb.rb:441:in
> `__file__'
>from home/hadoop/hbase193/bin/$_dot_dot_/bin/hirb.rb:-1:in
> `__file__'
>from home/hadoop/hbase193/bin/$_dot_dot_/bin/hirb.rb:-1:in `load'
>from org/jruby/Ruby.java:564:in `runScript'
>from org/jruby/Ruby.java:467:in `runNormally'
>from org/jruby/Ruby.java:340:in `runFromMain'
>from org/jruby/Main.java:214:in `run'
>from org/jruby/Main.java:100:in `run'
>from org/jruby/Main.java:84:in `main'
>from /home/hadoop/hbase193/bin/../bin/hirb.rb:346:in `scan'
>
>
> ===
>
> Is there an easy way around this problem?
>
>
>
>
> Billy Pearson-2 wrote:
> >
> > Yes that's what scanners are good for they will return all the
> > columns:lables combos for a row
> > What does the MR job stats say for rows processed for the maps and
> > reduces?
> >
> > Billy Pearson
> >
> >
> >
> > "llpind"  wrote in
> > message news:23967196.p...@talk.nabble.com...
> >>
> >> also,
> >>
> >> I think what we want is a way to wildcard everything after colFam1:
> >> (e.g.
> >> colFam1:*).  Is there a way to do this in HBase?
> >>
> >> This is assuming we dont know the column name, we want them all.
> >>
> >>
> >> llpind wrote:
> >>>
> >>> Thanks.
> >>>
> >>> Yea I've got that colFam for sure in the HBase table:
> >>>
> >>> {NAME => 'tableA', FAMILIES => [{NAME => 'colFam1', VERSIONS => '3',
> >>> COMPRESSION => 'NONE', LENGTH => '2147483647',
> >>>  TTL => '-1', IN_MEMORY => 'false', BLOCKCACHE => 'false'}, {NAME =>
> >>> 'colFam2', VERSIONS => '3', COMPRESSION =>
> >>>  'NONE', LENGTH => '2147483647', TTL => '-1', IN_MEMORY => 'false',
> >>> BLOCKCACHE => 'false'}]}
> >>>
> >>>
> >>> I've been trying to play with rowcounter, and not having much luck
> >>> either.
> >>>
> >>> I run the command:
> >>> hadoop19/bin/hadoop org.apache.hadoop.hbase.mapred.Driver rowcounter
> >>> /home/hadoop/dev/rowcounter7 tableA colFam1:
> >>>
> >>>
> >>> The map/reduce finishes just like it does with my own program, but with
> >>> all part files empty in /home/hadoop/dev/rowcounter7.
> >>>
> >>> Any Ideas?
> >>>
> >>>
> >>
> >> --
> >> View this message in context:

Re: Help with Map/Reduce program

2009-06-11 Thread Ryan Rawson
Also remember you might be able to convert to a tall table. Row keys can be
compound and you can do partial left matches on them. Eg:

Userid:timestamp:eventid

now you have a tall table. Do prefix matches on the userid you want and you
get results in chronological order.

You can build equivalent indexes in hbase as in sql. You may find a design
like this alieviates the need for extremely wide rows.

Good luck!

On Jun 11, 2009 11:44 AM, "Billy Pearson" 
wrote:

That might be a good idea but you might be able to redesign you layout of
the table
using a different key then the current one worth barnstorming.

Billy



"llpind"  wrote in message
news:23975432.p...@talk.nabble.com...

Sorry I forgot to mention the overflow then overflows into new row keys per
10,000 column entries ...


  1   2   3   4   5   >