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 <[email protected]> wrote: > 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 <[email protected] > >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 <[email protected]> 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 < > [email protected] > > > >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 <[email protected]> > > 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 < > > > [email protected] > > > > > >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: [email protected] > > > > > > homepage: http://germoglio.googlepages.com > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > Guilherme > > > > > > > > msn: [email protected] > > > > homepage: http://germoglio.googlepages.com > > > > > > > > > > > > > > > -- > > Guilherme > > > > msn: [email protected] > > homepage: http://germoglio.googlepages.com > > > -- Guilherme msn: [email protected] homepage: http://germoglio.googlepages.com
