|
Juhani Connolly
2012-03-19, 10:41
Ramkrishna.S.Vasudevan
2012-03-19, 11:03
Juhani Connolly
2012-03-19, 11:09
Mingjian Deng
2012-03-19, 11:56
Juhani Connolly
2012-03-19, 12:02
Christian Schäfer
2012-03-19, 12:21
Ramkrishna.S.Vasudevan
2012-03-19, 12:27
Juhani Connolly
2012-03-19, 12:31
Matt Corgan
2012-03-19, 16:55
Stack
2012-03-20, 19:27
Juhani Connolly
2012-03-21, 03:09
Mikael Sitruk
2012-03-21, 05:29
Juhani Connolly
2012-03-23, 07:40
Juhani Connolly
2012-03-23, 07:48
Matt Corgan
2012-03-26, 13:58
Mikael Sitruk
2012-03-26, 14:21
Stack
2012-03-26, 16:29
Stack
2012-03-26, 16:37
Stack
2012-03-26, 16:42
Juhani Connolly
2012-03-26, 16:48
Juhani Connolly
2012-03-26, 16:59
Juhani Connolly
2012-03-26, 17:02
Juhani Connolly
2012-03-26, 17:08
Todd Lipcon
2012-03-27, 01:43
Juhani Connolly
2012-03-27, 03:18
Juhani Connolly
2012-03-28, 08:27
Buckley,Ron
2012-03-28, 12:41
Stack
2012-03-28, 17:12
Buckley,Ron
2012-03-28, 17:56
Juhani Connolly
2012-03-29, 02:36
Alok Singh
2012-04-02, 17:15
Stack
2012-04-02, 17:41
Jonathan Hsieh
2012-04-03, 00:21
Juhani Connolly
2012-04-03, 02:52
Juhani Connolly
2012-04-03, 02:56
Juhani Connolly
2012-04-03, 03:02
Jonathan Hsieh
2012-04-03, 03:19
Stack
2012-04-03, 03:50
Jonathan Hsieh
2012-04-03, 16:56
Stack
2012-04-03, 17:42
Juhani Connolly
2012-04-05, 01:45
Ted Yu
2012-04-05, 02:43
Juhani Connolly
2012-04-05, 03:02
Todd Lipcon
2012-04-14, 03:02
Stack
2012-04-14, 04:06
Jonathan Hsieh
2012-04-14, 12:43
Todd Lipcon
2012-04-15, 06:28
|
-
0.92 and Read/writes not scalingJuhani Connolly 2012-03-19, 10:41
Hi,
We're running into a brick wall where our throughput numbers will not scale as we increase server counts both using custom inhouse tests and ycsb. We're using hbase 0.92 on hadoop 0.20.2(we also experience the same issues using 0.90 before switching our testing to this version). Our cluster consists of: - Namenode and hmaster on separate servers, 24 core, 64gb - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope to get this changed) We have adjusted our gc settings, and mslabs: <property> <name>hbase.hregion.memstore.mslab.enabled</name> <value>true</value> </property> <property> <name>hbase.hregion.memstore.mslab.chunksize</name> <value>2097152</value> </property> <property> <name>hbase.hregion.memstore.mslab.max.allocation</name> <value>1024768</value> </property> hdfs xceivers is set to 8192 We've experimented with a variety of handler counts for namenode, datanodes and regionservers with no changes in throughput. For testing with ycsb, we do the following each time(with nothing else using the cluster): - truncate test table - add a small amount of data, then split the table into 32 regions and call balancer from the shell. - load 10m rows - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) - do a 5:5 insert:update test with 10 million rows (23k/sec) - do a pure read test with 10 million rows (75k/sec) We have observed ganglia, iostat -d -x, iptraf, top, dstat and a variety of other diagnostic tools and network/io/cpu/memory as bottlenecks seem highly unlikely as none of them are ever seriously taxed. This leave me to assume this is some kind of locking issue? Delaying WAL flushes gives a small throughput bump but it doesn't scale. There also doesn't seem to be many figures around to compare ours to. We can get our throughput numbers higher with tricks like not writing the WAL or delaying flushes, batching requests, but nothing seems to scale with additional slaves. Could anyone provide guidance as to what may be preventing throughput figures from scaling as we increase our slave count?
-
RE: 0.92 and Read/writes not scalingRamkrishna.S.Vasudevan 2012-03-19, 11:03
Hi Juhani
Can you tell more on how the regions are balanced? Are you overloading only specific region server alone? Regards Ram > -----Original Message----- > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > Sent: Monday, March 19, 2012 4:11 PM > To: [EMAIL PROTECTED] > Subject: 0.92 and Read/writes not scaling > > Hi, > > We're running into a brick wall where our throughput numbers will not > scale as we increase server counts both using custom inhouse tests and > ycsb. > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > issues using 0.90 before switching our testing to this version). > > Our cluster consists of: > - Namenode and hmaster on separate servers, 24 core, 64gb > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > to get this changed) > > We have adjusted our gc settings, and mslabs: > > <property> > <name>hbase.hregion.memstore.mslab.enabled</name> > <value>true</value> > </property> > > <property> > <name>hbase.hregion.memstore.mslab.chunksize</name> > <value>2097152</value> > </property> > > <property> > <name>hbase.hregion.memstore.mslab.max.allocation</name> > <value>1024768</value> > </property> > > hdfs xceivers is set to 8192 > > We've experimented with a variety of handler counts for namenode, > datanodes and regionservers with no changes in throughput. > > For testing with ycsb, we do the following each time(with nothing else > using the cluster): > - truncate test table > - add a small amount of data, then split the table into 32 regions and > call balancer from the shell. > - load 10m rows > - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) > - do a 5:5 insert:update test with 10 million rows (23k/sec) > - do a pure read test with 10 million rows (75k/sec) > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a > variety of other diagnostic tools and network/io/cpu/memory as > bottlenecks seem highly unlikely as none of them are ever seriously > taxed. This leave me to assume this is some kind of locking issue? > Delaying WAL flushes gives a small throughput bump but it doesn't > scale. > > There also doesn't seem to be many figures around to compare ours to. > We can get our throughput numbers higher with tricks like not writing > the WAL or delaying flushes, batching requests, but nothing seems to > scale with additional slaves. > Could anyone provide guidance as to what may be preventing throughput > figures from scaling as we increase our slave count?
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-19, 11:09
Our custom tests are randomly distributed over 64 bit keys.
The ycsb tests use the zipfian request distribution(so its an uneven distribution to hit certain rows more frequently) Monitoring the web interface, most of the time the load is pretty even(though occasionally a region will briefly stop getting hits... Possibly due to compactions?). The following evidence is tested less rigorously, but when we didn't split tables in advance, we could get more or less the same write throughput on a single region. On Mon, Mar 19, 2012 at 8:03 PM, Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> wrote: > Hi Juhani > > Can you tell more on how the regions are balanced? > Are you overloading only specific region server alone? > > Regards > Ram > >> -----Original Message----- >> From: Juhani Connolly [mailto:[EMAIL PROTECTED]] >> Sent: Monday, March 19, 2012 4:11 PM >> To: [EMAIL PROTECTED] >> Subject: 0.92 and Read/writes not scaling >> >> Hi, >> >> We're running into a brick wall where our throughput numbers will not >> scale as we increase server counts both using custom inhouse tests and >> ycsb. >> >> We're using hbase 0.92 on hadoop 0.20.2(we also experience the same >> issues using 0.90 before switching our testing to this version). >> >> Our cluster consists of: >> - Namenode and hmaster on separate servers, 24 core, 64gb >> - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope >> to get this changed) >> >> We have adjusted our gc settings, and mslabs: >> >> <property> >> <name>hbase.hregion.memstore.mslab.enabled</name> >> <value>true</value> >> </property> >> >> <property> >> <name>hbase.hregion.memstore.mslab.chunksize</name> >> <value>2097152</value> >> </property> >> >> <property> >> <name>hbase.hregion.memstore.mslab.max.allocation</name> >> <value>1024768</value> >> </property> >> >> hdfs xceivers is set to 8192 >> >> We've experimented with a variety of handler counts for namenode, >> datanodes and regionservers with no changes in throughput. >> >> For testing with ycsb, we do the following each time(with nothing else >> using the cluster): >> - truncate test table >> - add a small amount of data, then split the table into 32 regions and >> call balancer from the shell. >> - load 10m rows >> - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) >> - do a 5:5 insert:update test with 10 million rows (23k/sec) >> - do a pure read test with 10 million rows (75k/sec) >> >> We have observed ganglia, iostat -d -x, iptraf, top, dstat and a >> variety of other diagnostic tools and network/io/cpu/memory as >> bottlenecks seem highly unlikely as none of them are ever seriously >> taxed. This leave me to assume this is some kind of locking issue? >> Delaying WAL flushes gives a small throughput bump but it doesn't >> scale. >> >> There also doesn't seem to be many figures around to compare ours to. >> We can get our throughput numbers higher with tricks like not writing >> the WAL or delaying flushes, batching requests, but nothing seems to >> scale with additional slaves. >> Could anyone provide guidance as to what may be preventing throughput >> figures from scaling as we increase our slave count? >
-
Re: 0.92 and Read/writes not scalingMingjian Deng 2012-03-19, 11:56
@Juhani:
How many clients did you test? Maybe the bottleneck was client? 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > Hi Juhani > > Can you tell more on how the regions are balanced? > Are you overloading only specific region server alone? > > Regards > Ram > > > -----Original Message----- > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > > Sent: Monday, March 19, 2012 4:11 PM > > To: [EMAIL PROTECTED] > > Subject: 0.92 and Read/writes not scaling > > > > Hi, > > > > We're running into a brick wall where our throughput numbers will not > > scale as we increase server counts both using custom inhouse tests and > > ycsb. > > > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > > issues using 0.90 before switching our testing to this version). > > > > Our cluster consists of: > > - Namenode and hmaster on separate servers, 24 core, 64gb > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > > to get this changed) > > > > We have adjusted our gc settings, and mslabs: > > > > <property> > > <name>hbase.hregion.memstore.mslab.enabled</name> > > <value>true</value> > > </property> > > > > <property> > > <name>hbase.hregion.memstore.mslab.chunksize</name> > > <value>2097152</value> > > </property> > > > > <property> > > <name>hbase.hregion.memstore.mslab.max.allocation</name> > > <value>1024768</value> > > </property> > > > > hdfs xceivers is set to 8192 > > > > We've experimented with a variety of handler counts for namenode, > > datanodes and regionservers with no changes in throughput. > > > > For testing with ycsb, we do the following each time(with nothing else > > using the cluster): > > - truncate test table > > - add a small amount of data, then split the table into 32 regions and > > call balancer from the shell. > > - load 10m rows > > - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) > > - do a 5:5 insert:update test with 10 million rows (23k/sec) > > - do a pure read test with 10 million rows (75k/sec) > > > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a > > variety of other diagnostic tools and network/io/cpu/memory as > > bottlenecks seem highly unlikely as none of them are ever seriously > > taxed. This leave me to assume this is some kind of locking issue? > > Delaying WAL flushes gives a small throughput bump but it doesn't > > scale. > > > > There also doesn't seem to be many figures around to compare ours to. > > We can get our throughput numbers higher with tricks like not writing > > the WAL or delaying flushes, batching requests, but nothing seems to > > scale with additional slaves. > > Could anyone provide guidance as to what may be preventing throughput > > figures from scaling as we increase our slave count? > >
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-19, 12:02
I was concerned that may be the case too, which is why we ran the ycsb
tests in addition to our application specific and general performance tests. checking profiles of the execution just showed the vast majority of time spent waiting for responses. these were all run with 400 threads(though we tried more/less just in case) 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > @Juhani: > How many clients did you test? Maybe the bottleneck was client? > > 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > > > Hi Juhani > > > > Can you tell more on how the regions are balanced? > > Are you overloading only specific region server alone? > > > > Regards > > Ram > > > > > -----Original Message----- > > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > > > Sent: Monday, March 19, 2012 4:11 PM > > > To: [EMAIL PROTECTED] > > > Subject: 0.92 and Read/writes not scaling > > > > > > Hi, > > > > > > We're running into a brick wall where our throughput numbers will not > > > scale as we increase server counts both using custom inhouse tests and > > > ycsb. > > > > > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > > > issues using 0.90 before switching our testing to this version). > > > > > > Our cluster consists of: > > > - Namenode and hmaster on separate servers, 24 core, 64gb > > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > > > to get this changed) > > > > > > We have adjusted our gc settings, and mslabs: > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.enabled</name> > > > <value>true</value> > > > </property> > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.chunksize</name> > > > <value>2097152</value> > > > </property> > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.max.allocation</name> > > > <value>1024768</value> > > > </property> > > > > > > hdfs xceivers is set to 8192 > > > > > > We've experimented with a variety of handler counts for namenode, > > > datanodes and regionservers with no changes in throughput. > > > > > > For testing with ycsb, we do the following each time(with nothing else > > > using the cluster): > > > - truncate test table > > > - add a small amount of data, then split the table into 32 regions and > > > call balancer from the shell. > > > - load 10m rows > > > - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) > > > - do a 5:5 insert:update test with 10 million rows (23k/sec) > > > - do a pure read test with 10 million rows (75k/sec) > > > > > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a > > > variety of other diagnostic tools and network/io/cpu/memory as > > > bottlenecks seem highly unlikely as none of them are ever seriously > > > taxed. This leave me to assume this is some kind of locking issue? > > > Delaying WAL flushes gives a small throughput bump but it doesn't > > > scale. > > > > > > There also doesn't seem to be many figures around to compare ours to. > > > We can get our throughput numbers higher with tricks like not writing > > > the WAL or delaying flushes, batching requests, but nothing seems to > > > scale with additional slaves. > > > Could anyone provide guidance as to what may be preventing throughput > > > figures from scaling as we increase our slave count? > > > > >
-
Re: 0.92 and Read/writes not scalingChristian Schäfer 2012-03-19, 12:21
referring to my experiences I expect the client to be the bottleneck, too.
So try to increase the count of client-machines (not client threads) each with its own unshared network interface. In my case I could double write throughput by doubling client machine count with a much smaller system than yours (5 machines, 4gigs RAM each). Good Luck Chris ________________________________ Von: Juhani Connolly <[EMAIL PROTECTED]> An: [EMAIL PROTECTED] Gesendet: 13:02 Montag, 19.März 2012 Betreff: Re: 0.92 and Read/writes not scaling I was concerned that may be the case too, which is why we ran the ycsb tests in addition to our application specific and general performance tests. checking profiles of the execution just showed the vast majority of time spent waiting for responses. these were all run with 400 threads(though we tried more/less just in case) 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > @Juhani: > How many clients did you test? Maybe the bottleneck was client? > > 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > > > Hi Juhani > > > > Can you tell more on how the regions are balanced? > > Are you overloading only specific region server alone? > > > > Regards > > Ram > > > > > -----Original Message----- > > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > > > Sent: Monday, March 19, 2012 4:11 PM > > > To: [EMAIL PROTECTED] > > > Subject: 0.92 and Read/writes not scaling > > > > > > Hi, > > > > > > We're running into a brick wall where our throughput numbers will not > > > scale as we increase server counts both using custom inhouse tests and > > > ycsb. > > > > > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > > > issues using 0.90 before switching our testing to this version). > > > > > > Our cluster consists of: > > > - Namenode and hmaster on separate servers, 24 core, 64gb > > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > > > to get this changed) > > > > > > We have adjusted our gc settings, and mslabs: > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.enabled</name> > > > <value>true</value> > > > </property> > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.chunksize</name> > > > <value>2097152</value> > > > </property> > > > > > > <property> > > > <name>hbase.hregion.memstore.mslab.max.allocation</name> > > > <value>1024768</value> > > > </property> > > > > > > hdfs xceivers is set to 8192 > > > > > > We've experimented with a variety of handler counts for namenode, > > > datanodes and regionservers with no changes in throughput. > > > > > > For testing with ycsb, we do the following each time(with nothing else > > > using the cluster): > > > - truncate test table > > > - add a small amount of data, then split the table into 32 regions and > > > call balancer from the shell. > > > - load 10m rows > > > - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) > > > - do a 5:5 insert:update test with 10 million rows (23k/sec) > > > - do a pure read test with 10 million rows (75k/sec) > > > > > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a > > > variety of other diagnostic tools and network/io/cpu/memory as > > > bottlenecks seem highly unlikely as none of them are ever seriously > > > taxed. This leave me to assume this is some kind of locking issue? > > > Delaying WAL flushes gives a small throughput bump but it doesn't > > > scale. > > > > > > There also doesn't seem to be many figures around to compare ours to. > > > We can get our throughput numbers higher with tricks like not writing > > > the WAL or delaying flushes, batching requests, but nothing seems to > > > scale with additional slaves. > > > Could anyone provide guidance as to what may be preventing throughput > > > figures from scaling as we increase our slave count? > > > > >
-
RE: 0.92 and Read/writes not scalingRamkrishna.S.Vasudevan 2012-03-19, 12:27
Hi
In our experience rather than increasing threads increase the number of clients. Increasing the client number has given us better throughput. Regards Ram > -----Original Message----- > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > Sent: Monday, March 19, 2012 5:33 PM > To: [EMAIL PROTECTED] > Subject: Re: 0.92 and Read/writes not scaling > > I was concerned that may be the case too, which is why we ran the ycsb > tests in addition to our application specific and general performance > tests. checking profiles of the execution just showed the vast majority > of > time spent waiting for responses. these were all run with 400 > threads(though we tried more/less just in case) > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > > > @Juhani: > > How many clients did you test? Maybe the bottleneck was client? > > > > 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > > > > > Hi Juhani > > > > > > Can you tell more on how the regions are balanced? > > > Are you overloading only specific region server alone? > > > > > > Regards > > > Ram > > > > > > > -----Original Message----- > > > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > > > > Sent: Monday, March 19, 2012 4:11 PM > > > > To: [EMAIL PROTECTED] > > > > Subject: 0.92 and Read/writes not scaling > > > > > > > > Hi, > > > > > > > > We're running into a brick wall where our throughput numbers will > not > > > > scale as we increase server counts both using custom inhouse > tests and > > > > ycsb. > > > > > > > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the > same > > > > issues using 0.90 before switching our testing to this version). > > > > > > > > Our cluster consists of: > > > > - Namenode and hmaster on separate servers, 24 core, 64gb > > > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb > disks(hope > > > > to get this changed) > > > > > > > > We have adjusted our gc settings, and mslabs: > > > > > > > > <property> > > > > <name>hbase.hregion.memstore.mslab.enabled</name> > > > > <value>true</value> > > > > </property> > > > > > > > > <property> > > > > <name>hbase.hregion.memstore.mslab.chunksize</name> > > > > <value>2097152</value> > > > > </property> > > > > > > > > <property> > > > > <name>hbase.hregion.memstore.mslab.max.allocation</name> > > > > <value>1024768</value> > > > > </property> > > > > > > > > hdfs xceivers is set to 8192 > > > > > > > > We've experimented with a variety of handler counts for namenode, > > > > datanodes and regionservers with no changes in throughput. > > > > > > > > For testing with ycsb, we do the following each time(with nothing > else > > > > using the cluster): > > > > - truncate test table > > > > - add a small amount of data, then split the table into 32 > regions and > > > > call balancer from the shell. > > > > - load 10m rows > > > > - do a 1:2:7 insert:update:read test with 10million rows > (64k/sec) > > > > - do a 5:5 insert:update test with 10 million rows (23k/sec) > > > > - do a pure read test with 10 million rows (75k/sec) > > > > > > > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a > > > > variety of other diagnostic tools and network/io/cpu/memory as > > > > bottlenecks seem highly unlikely as none of them are ever > seriously > > > > taxed. This leave me to assume this is some kind of locking > issue? > > > > Delaying WAL flushes gives a small throughput bump but it doesn't > > > > scale. > > > > > > > > There also doesn't seem to be many figures around to compare ours > to. > > > > We can get our throughput numbers higher with tricks like not > writing > > > > the WAL or delaying flushes, batching requests, but nothing seems > to > > > > scale with additional slaves. > > > > Could anyone provide guidance as to what may be preventing > throughput > > > > figures from scaling as we increase our slave count? > > > > > > > >
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-19, 12:31
Actually we did try running off two machines both running our own
tests in parallel. Unfortunately the results were a split that results in the same total throughput. We also did the same thing with iperf running from each machine to another machine, indicating 800Mb additional throughput between each pair of machines. However we didn't try these tests very thoroughly so I will revisit them as soon as I get back to the office, thanks. On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer <[EMAIL PROTECTED]> wrote: > referring to my experiences I expect the client to be the bottleneck, too. > > So try to increase the count of client-machines (not client threads) each with its own unshared network interface. > > In my case I could double write throughput by doubling client machine count with a much smaller system than yours (5 machines, 4gigs RAM each). > > Good Luck > Chris > > > > ________________________________ > Von: Juhani Connolly <[EMAIL PROTECTED]> > An: [EMAIL PROTECTED] > Gesendet: 13:02 Montag, 19.März 2012 > Betreff: Re: 0.92 and Read/writes not scaling > > I was concerned that may be the case too, which is why we ran the ycsb > tests in addition to our application specific and general performance > tests. checking profiles of the execution just showed the vast majority of > time spent waiting for responses. these were all run with 400 > threads(though we tried more/less just in case) > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > >> @Juhani: >> How many clients did you test? Maybe the bottleneck was client? >> >> 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> >> >> > Hi Juhani >> > >> > Can you tell more on how the regions are balanced? >> > Are you overloading only specific region server alone? >> > >> > Regards >> > Ram >> > >> > > -----Original Message----- >> > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] >> > > Sent: Monday, March 19, 2012 4:11 PM >> > > To: [EMAIL PROTECTED] >> > > Subject: 0.92 and Read/writes not scaling >> > > >> > > Hi, >> > > >> > > We're running into a brick wall where our throughput numbers will not >> > > scale as we increase server counts both using custom inhouse tests and >> > > ycsb. >> > > >> > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same >> > > issues using 0.90 before switching our testing to this version). >> > > >> > > Our cluster consists of: >> > > - Namenode and hmaster on separate servers, 24 core, 64gb >> > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope >> > > to get this changed) >> > > >> > > We have adjusted our gc settings, and mslabs: >> > > >> > > <property> >> > > <name>hbase.hregion.memstore.mslab.enabled</name> >> > > <value>true</value> >> > > </property> >> > > >> > > <property> >> > > <name>hbase.hregion.memstore.mslab.chunksize</name> >> > > <value>2097152</value> >> > > </property> >> > > >> > > <property> >> > > <name>hbase.hregion.memstore.mslab.max.allocation</name> >> > > <value>1024768</value> >> > > </property> >> > > >> > > hdfs xceivers is set to 8192 >> > > >> > > We've experimented with a variety of handler counts for namenode, >> > > datanodes and regionservers with no changes in throughput. >> > > >> > > For testing with ycsb, we do the following each time(with nothing else >> > > using the cluster): >> > > - truncate test table >> > > - add a small amount of data, then split the table into 32 regions and >> > > call balancer from the shell. >> > > - load 10m rows >> > > - do a 1:2:7 insert:update:read test with 10million rows (64k/sec) >> > > - do a 5:5 insert:update test with 10 million rows (23k/sec) >> > > - do a pure read test with 10 million rows (75k/sec) >> > > >> > > We have observed ganglia, iostat -d -x, iptraf, top, dstat and a >> > > variety of other diagnostic tools and network/io/cpu/memory as >> > > bottlenecks seem highly unlikely as none of them are ever seriously >> > > taxed. This leave me to assume this is some kind of locking issue?
-
Re: 0.92 and Read/writes not scalingMatt Corgan 2012-03-19, 16:55
I'd be curious to see what happens if you split the table into 1 region per
CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 memstore which is a ConcurrentSkipListMap, and you're currently hitting each CSLM with 8 cores which might be too contentious. Normally in production you would want multiple memstores per CPU core. On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > Actually we did try running off two machines both running our own > tests in parallel. Unfortunately the results were a split that results > in the same total throughput. We also did the same thing with iperf > running from each machine to another machine, indicating 800Mb > additional throughput between each pair of machines. > However we didn't try these tests very thoroughly so I will revisit > them as soon as I get back to the office, thanks. > > On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer <[EMAIL PROTECTED]> > wrote: > > referring to my experiences I expect the client to be the bottleneck, > too. > > > > So try to increase the count of client-machines (not client threads) > each with its own unshared network interface. > > > > In my case I could double write throughput by doubling client machine > count with a much smaller system than yours (5 machines, 4gigs RAM each). > > > > Good Luck > > Chris > > > > > > > > ________________________________ > > Von: Juhani Connolly <[EMAIL PROTECTED]> > > An: [EMAIL PROTECTED] > > Gesendet: 13:02 Montag, 19.März 2012 > > Betreff: Re: 0.92 and Read/writes not scaling > > > > I was concerned that may be the case too, which is why we ran the ycsb > > tests in addition to our application specific and general performance > > tests. checking profiles of the execution just showed the vast majority > of > > time spent waiting for responses. these were all run with 400 > > threads(though we tried more/less just in case) > > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > > > >> @Juhani: > >> How many clients did you test? Maybe the bottleneck was client? > >> > >> 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > >> > >> > Hi Juhani > >> > > >> > Can you tell more on how the regions are balanced? > >> > Are you overloading only specific region server alone? > >> > > >> > Regards > >> > Ram > >> > > >> > > -----Original Message----- > >> > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > >> > > Sent: Monday, March 19, 2012 4:11 PM > >> > > To: [EMAIL PROTECTED] > >> > > Subject: 0.92 and Read/writes not scaling > >> > > > >> > > Hi, > >> > > > >> > > We're running into a brick wall where our throughput numbers will > not > >> > > scale as we increase server counts both using custom inhouse tests > and > >> > > ycsb. > >> > > > >> > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > >> > > issues using 0.90 before switching our testing to this version). > >> > > > >> > > Our cluster consists of: > >> > > - Namenode and hmaster on separate servers, 24 core, 64gb > >> > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > >> > > to get this changed) > >> > > > >> > > We have adjusted our gc settings, and mslabs: > >> > > > >> > > <property> > >> > > <name>hbase.hregion.memstore.mslab.enabled</name> > >> > > <value>true</value> > >> > > </property> > >> > > > >> > > <property> > >> > > <name>hbase.hregion.memstore.mslab.chunksize</name> > >> > > <value>2097152</value> > >> > > </property> > >> > > > >> > > <property> > >> > > <name>hbase.hregion.memstore.mslab.max.allocation</name> > >> > > <value>1024768</value> > >> > > </property> > >> > > > >> > > hdfs xceivers is set to 8192 > >> > > > >> > > We've experimented with a variety of handler counts for namenode, > >> > > datanodes and regionservers with no changes in throughput. > >> > > > >> > > For testing with ycsb, we do the following each time(with nothing > else > >> > > using the cluster): > >> > > - truncate test table
-
Re: 0.92 and Read/writes not scalingStack 2012-03-20, 19:27
On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote:
> Actually we did try running off two machines both running our own > tests in parallel. Unfortunately the results were a split that results > in the same total throughput. We also did the same thing with iperf > running from each machine to another machine, indicating 800Mb > additional throughput between each pair of machines. > However we didn't try these tests very thoroughly so I will revisit > them as soon as I get back to the office, thanks. > Let us know how it goes Juhani. There is no hbase bottleneck that we know other than the physical ones of net and disk, etc., so adding machines should be upping what you can handle. St.Ack
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-21, 03:09
Hi Matt,
this is something we haven't tested much, we were always running with about 32 regions which gave enough coverage for an even spread over all machines. I will run our tests with enough regions per server to cover all cores and get back to the ml On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: > I'd be curious to see what happens if you split the table into 1 region per > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 > memstore which is a ConcurrentSkipListMap, and you're currently hitting > each CSLM with 8 cores which might be too contentious. Normally in > production you would want multiple memstores per CPU core. > > > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > >> Actually we did try running off two machines both running our own >> tests in parallel. Unfortunately the results were a split that results >> in the same total throughput. We also did the same thing with iperf >> running from each machine to another machine, indicating 800Mb >> additional throughput between each pair of machines. >> However we didn't try these tests very thoroughly so I will revisit >> them as soon as I get back to the office, thanks. >> >> On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer <[EMAIL PROTECTED]> >> wrote: >> > referring to my experiences I expect the client to be the bottleneck, >> too. >> > >> > So try to increase the count of client-machines (not client threads) >> each with its own unshared network interface. >> > >> > In my case I could double write throughput by doubling client machine >> count with a much smaller system than yours (5 machines, 4gigs RAM each). >> > >> > Good Luck >> > Chris >> > >> > >> > >> > ________________________________ >> > Von: Juhani Connolly <[EMAIL PROTECTED]> >> > An: [EMAIL PROTECTED] >> > Gesendet: 13:02 Montag, 19.März 2012 >> > Betreff: Re: 0.92 and Read/writes not scaling >> > >> > I was concerned that may be the case too, which is why we ran the ycsb >> > tests in addition to our application specific and general performance >> > tests. checking profiles of the execution just showed the vast majority >> of >> > time spent waiting for responses. these were all run with 400 >> > threads(though we tried more/less just in case) >> > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: >> > >> >> @Juhani: >> >> How many clients did you test? Maybe the bottleneck was client? >> >> >> >> 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> >> >> >> >> > Hi Juhani >> >> > >> >> > Can you tell more on how the regions are balanced? >> >> > Are you overloading only specific region server alone? >> >> > >> >> > Regards >> >> > Ram >> >> > >> >> > > -----Original Message----- >> >> > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] >> >> > > Sent: Monday, March 19, 2012 4:11 PM >> >> > > To: [EMAIL PROTECTED] >> >> > > Subject: 0.92 and Read/writes not scaling >> >> > > >> >> > > Hi, >> >> > > >> >> > > We're running into a brick wall where our throughput numbers will >> not >> >> > > scale as we increase server counts both using custom inhouse tests >> and >> >> > > ycsb. >> >> > > >> >> > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same >> >> > > issues using 0.90 before switching our testing to this version). >> >> > > >> >> > > Our cluster consists of: >> >> > > - Namenode and hmaster on separate servers, 24 core, 64gb >> >> > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope >> >> > > to get this changed) >> >> > > >> >> > > We have adjusted our gc settings, and mslabs: >> >> > > >> >> > > <property> >> >> > > <name>hbase.hregion.memstore.mslab.enabled</name> >> >> > > <value>true</value> >> >> > > </property> >> >> > > >> >> > > <property> >> >> > > <name>hbase.hregion.memstore.mslab.chunksize</name> >> >> > > <value>2097152</value> >> >> > > </property> >> >> > > >> >> > > <property> >> >> > > <name>hbase.hregion.memstore.mslab.max.allocation</name>
-
Re: 0.92 and Read/writes not scalingMikael Sitruk 2012-03-21, 05:29
Juhani,
Can you look at the storefiles and tell how they behave during the test? What is the size of the data you insert/update? Mikael On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: > Hi Matt, > > this is something we haven't tested much, we were always running with > about 32 regions which gave enough coverage for an even spread over > all machines. > I will run our tests with enough regions per server to cover all cores > and get back to the ml > > On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: > > I'd be curious to see what happens if you split the table into 1 region > per > > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 > > memstore which is a ConcurrentSkipListMap, and you're currently hitting > > each CSLM with 8 cores which might be too contentious. Normally in > > production you would want multiple memstores per CPU core. > > > > > > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> > wrote: > > > >> Actually we did try running off two machines both running our own > >> tests in parallel. Unfortunately the results were a split that results > >> in the same total throughput. We also did the same thing with iperf > >> running from each machine to another machine, indicating 800Mb > >> additional throughput between each pair of machines. > >> However we didn't try these tests very thoroughly so I will revisit > >> them as soon as I get back to the office, thanks. > >> > >> On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer < > [EMAIL PROTECTED]> > >> wrote: > >> > referring to my experiences I expect the client to be the bottleneck, > >> too. > >> > > >> > So try to increase the count of client-machines (not client threads) > >> each with its own unshared network interface. > >> > > >> > In my case I could double write throughput by doubling client machine > >> count with a much smaller system than yours (5 machines, 4gigs RAM > each). > >> > > >> > Good Luck > >> > Chris > >> > > >> > > >> > > >> > ________________________________ > >> > Von: Juhani Connolly <[EMAIL PROTECTED]> > >> > An: [EMAIL PROTECTED] > >> > Gesendet: 13:02 Montag, 19.März 2012 > >> > Betreff: Re: 0.92 and Read/writes not scaling > >> > > >> > I was concerned that may be the case too, which is why we ran the ycsb > >> > tests in addition to our application specific and general performance > >> > tests. checking profiles of the execution just showed the vast > majority > >> of > >> > time spent waiting for responses. these were all run with 400 > >> > threads(though we tried more/less just in case) > >> > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: > >> > > >> >> @Juhani: > >> >> How many clients did you test? Maybe the bottleneck was client? > >> >> > >> >> 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> > >> >> > >> >> > Hi Juhani > >> >> > > >> >> > Can you tell more on how the regions are balanced? > >> >> > Are you overloading only specific region server alone? > >> >> > > >> >> > Regards > >> >> > Ram > >> >> > > >> >> > > -----Original Message----- > >> >> > > From: Juhani Connolly [mailto:[EMAIL PROTECTED]] > >> >> > > Sent: Monday, March 19, 2012 4:11 PM > >> >> > > To: [EMAIL PROTECTED] > >> >> > > Subject: 0.92 and Read/writes not scaling > >> >> > > > >> >> > > Hi, > >> >> > > > >> >> > > We're running into a brick wall where our throughput numbers will > >> not > >> >> > > scale as we increase server counts both using custom inhouse > tests > >> and > >> >> > > ycsb. > >> >> > > > >> >> > > We're using hbase 0.92 on hadoop 0.20.2(we also experience the > same > >> >> > > issues using 0.90 before switching our testing to this version). > >> >> > > > >> >> > > Our cluster consists of: > >> >> > > - Namenode and hmaster on separate servers, 24 core, 64gb > >> >> > > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb > disks(hope > >> >> > > to get this changed) > >> >> > > > >> >> > > We have adjusted our gc settings, and mslabs:
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-23, 07:40
Status update:
- We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were using 0.20.2 series) - Did the tests now with 256/512 regions, the numbers do appear to scale which is good. BUT, our write throughput has gone in the dump. If we disable wal writes, we still get nearly 40,000 a second, but with it on, we're lucky to get more than 12,000. Before we were getting as high as 70,000 grouping puts together. Have set up log collection, and am not finding anything unusual in the logs. Mikael: One of the tests is the ycsb one where we just let it choose the size. Our own custom test has a configurable size, but we have been testing with entries that are 128-256 bytes per entry, as this is what we expect in our application. What exactly should we be looking at with the storefiles? On Wed, Mar 21, 2012 at 2:29 PM, Mikael Sitruk <[EMAIL PROTECTED]> wrote: > Juhani, > Can you look at the storefiles and tell how they behave during the test? > What is the size of the data you insert/update? > Mikael > On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: > >> Hi Matt, >> >> this is something we haven't tested much, we were always running with >> about 32 regions which gave enough coverage for an even spread over >> all machines. >> I will run our tests with enough regions per server to cover all cores >> and get back to the ml >> >> On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: >> > I'd be curious to see what happens if you split the table into 1 region >> per >> > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 >> > memstore which is a ConcurrentSkipListMap, and you're currently hitting >> > each CSLM with 8 cores which might be too contentious. Normally in >> > production you would want multiple memstores per CPU core. >> > >> > >> > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> >> wrote: >> > >> >> Actually we did try running off two machines both running our own >> >> tests in parallel. Unfortunately the results were a split that results >> >> in the same total throughput. We also did the same thing with iperf >> >> running from each machine to another machine, indicating 800Mb >> >> additional throughput between each pair of machines. >> >> However we didn't try these tests very thoroughly so I will revisit >> >> them as soon as I get back to the office, thanks. >> >> >> >> On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer < >> [EMAIL PROTECTED]> >> >> wrote: >> >> > referring to my experiences I expect the client to be the bottleneck, >> >> too. >> >> > >> >> > So try to increase the count of client-machines (not client threads) >> >> each with its own unshared network interface. >> >> > >> >> > In my case I could double write throughput by doubling client machine >> >> count with a much smaller system than yours (5 machines, 4gigs RAM >> each). >> >> > >> >> > Good Luck >> >> > Chris >> >> > >> >> > >> >> > >> >> > ________________________________ >> >> > Von: Juhani Connolly <[EMAIL PROTECTED]> >> >> > An: [EMAIL PROTECTED] >> >> > Gesendet: 13:02 Montag, 19.März 2012 >> >> > Betreff: Re: 0.92 and Read/writes not scaling >> >> > >> >> > I was concerned that may be the case too, which is why we ran the ycsb >> >> > tests in addition to our application specific and general performance >> >> > tests. checking profiles of the execution just showed the vast >> majority >> >> of >> >> > time spent waiting for responses. these were all run with 400 >> >> > threads(though we tried more/less just in case) >> >> > 2012/03/19 20:57 "Mingjian Deng" <[EMAIL PROTECTED]>: >> >> > >> >> >> @Juhani: >> >> >> How many clients did you test? Maybe the bottleneck was client? >> >> >> >> >> >> 2012/3/19 Ramkrishna.S.Vasudevan <[EMAIL PROTECTED]> >> >> >> >> >> >> > Hi Juhani >> >> >> > >> >> >> > Can you tell more on how the regions are balanced? >> >> >> > Are you overloading only specific region server alone? >> >> >> >
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-23, 07:48
Also, the latency on requests is extremely long. If we group them into
sets of 10 puts(128-256 bytes each) before flushing the client table, latency is over 1 second. We get entries like this in our logs: 22:17:51,010 WARN org.apache.hadoop.ipc.HBaseServer: (responseTooSlow): {"processingtimems":16692,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@65312e3b), rpc version=1, client version=29, methodsFingerPrint=54742778","client":"10.172.109.3:42725","starttimems":1332335854317,"queuetimems":6387,"class":"HRegionServer","responsesize":0,"method":"multi"} Any suggestions as to where we should be digging? On Fri, Mar 23, 2012 at 4:40 PM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > Status update: > > - We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were > using 0.20.2 series) > - Did the tests now with 256/512 regions, the numbers do appear to > scale which is good. > > BUT, our write throughput has gone in the dump. If we disable wal > writes, we still get nearly 40,000 a second, but with it on, we're > lucky to get more than 12,000. Before we were getting as high as > 70,000 grouping puts together. Have set up log collection, and am not > finding anything unusual in the logs. > > Mikael: One of the tests is the ycsb one where we just let it choose > the size. Our own custom test has a configurable size, but we have > been testing with entries that are 128-256 bytes per entry, as this is > what we expect in our application. What exactly should we be looking > at with the storefiles? > > On Wed, Mar 21, 2012 at 2:29 PM, Mikael Sitruk <[EMAIL PROTECTED]> wrote: >> Juhani, >> Can you look at the storefiles and tell how they behave during the test? >> What is the size of the data you insert/update? >> Mikael >> On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: >> >>> Hi Matt, >>> >>> this is something we haven't tested much, we were always running with >>> about 32 regions which gave enough coverage for an even spread over >>> all machines. >>> I will run our tests with enough regions per server to cover all cores >>> and get back to the ml >>> >>> On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: >>> > I'd be curious to see what happens if you split the table into 1 region >>> per >>> > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 >>> > memstore which is a ConcurrentSkipListMap, and you're currently hitting >>> > each CSLM with 8 cores which might be too contentious. Normally in >>> > production you would want multiple memstores per CPU core. >>> > >>> > >>> > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> >>> wrote: >>> > >>> >> Actually we did try running off two machines both running our own >>> >> tests in parallel. Unfortunately the results were a split that results >>> >> in the same total throughput. We also did the same thing with iperf >>> >> running from each machine to another machine, indicating 800Mb >>> >> additional throughput between each pair of machines. >>> >> However we didn't try these tests very thoroughly so I will revisit >>> >> them as soon as I get back to the office, thanks. >>> >> >>> >> On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer < >>> [EMAIL PROTECTED]> >>> >> wrote: >>> >> > referring to my experiences I expect the client to be the bottleneck, >>> >> too. >>> >> > >>> >> > So try to increase the count of client-machines (not client threads) >>> >> each with its own unshared network interface. >>> >> > >>> >> > In my case I could double write throughput by doubling client machine >>> >> count with a much smaller system than yours (5 machines, 4gigs RAM >>> each). >>> >> > >>> >> > Good Luck >>> >> > Chris >>> >> > >>> >> > >>> >> > >>> >> > ________________________________ >>> >> > Von: Juhani Connolly <[EMAIL PROTECTED]> >>> >> > An: [EMAIL PROTECTED] >>> >> > Gesendet: 13:02 Montag, 19.März 2012 >>> >> > Betreff: Re: 0.92 and Read/writes not scaling >>> >> > >>>
-
Re: 0.92 and Read/writes not scalingMatt Corgan 2012-03-26, 13:58
When you increased regions on your previous test, did it start maxing out
CPU? What improvement did you see? Have you tried increasing the memstore flush size to something like 512MB? Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for a disabled WAL i think, especially with batch size of 10. If you increase write batch size to 1000 how much does your write throughput increase? On Fri, Mar 23, 2012 at 3:48 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > Also, the latency on requests is extremely long. If we group them into > sets of 10 puts(128-256 bytes each) before flushing the client table, > latency is over 1 second. > > We get entries like this in our logs: > 22:17:51,010 WARN org.apache.hadoop.ipc.HBaseServer: > (responseTooSlow): > > {"processingtimems":16692,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@65312e3b > ), > rpc version=1, client version=29, > methodsFingerPrint=54742778","client":"10.172.109.3:42725 > ","starttimems":1332335854317,"queuetimems":6387,"class":"HRegionServer","responsesize":0,"method":"multi"} > > Any suggestions as to where we should be digging? > > On Fri, Mar 23, 2012 at 4:40 PM, Juhani Connolly <[EMAIL PROTECTED]> > wrote: > > Status update: > > > > - We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were > > using 0.20.2 series) > > - Did the tests now with 256/512 regions, the numbers do appear to > > scale which is good. > > > > BUT, our write throughput has gone in the dump. If we disable wal > > writes, we still get nearly 40,000 a second, but with it on, we're > > lucky to get more than 12,000. Before we were getting as high as > > 70,000 grouping puts together. Have set up log collection, and am not > > finding anything unusual in the logs. > > > > Mikael: One of the tests is the ycsb one where we just let it choose > > the size. Our own custom test has a configurable size, but we have > > been testing with entries that are 128-256 bytes per entry, as this is > > what we expect in our application. What exactly should we be looking > > at with the storefiles? > > > > On Wed, Mar 21, 2012 at 2:29 PM, Mikael Sitruk <[EMAIL PROTECTED]> > wrote: > >> Juhani, > >> Can you look at the storefiles and tell how they behave during the test? > >> What is the size of the data you insert/update? > >> Mikael > >> On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: > >> > >>> Hi Matt, > >>> > >>> this is something we haven't tested much, we were always running with > >>> about 32 regions which gave enough coverage for an even spread over > >>> all machines. > >>> I will run our tests with enough regions per server to cover all cores > >>> and get back to the ml > >>> > >>> On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> > wrote: > >>> > I'd be curious to see what happens if you split the table into 1 > region > >>> per > >>> > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 > >>> > memstore which is a ConcurrentSkipListMap, and you're currently > hitting > >>> > each CSLM with 8 cores which might be too contentious. Normally in > >>> > production you would want multiple memstores per CPU core. > >>> > > >>> > > >>> > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]> > >>> wrote: > >>> > > >>> >> Actually we did try running off two machines both running our own > >>> >> tests in parallel. Unfortunately the results were a split that > results > >>> >> in the same total throughput. We also did the same thing with iperf > >>> >> running from each machine to another machine, indicating 800Mb > >>> >> additional throughput between each pair of machines. > >>> >> However we didn't try these tests very thoroughly so I will revisit > >>> >> them as soon as I get back to the office, thanks. > >>> >> > >>> >> On Mon, Mar 19, 2012 at 9:21 PM, Christian Schäfer < > >>> [EMAIL PROTECTED]> > >>> >> wrote: > >>> >> > referring to my experiences I expect the client to be the > bottleneck,
-
Re: 0.92 and Read/writes not scalingMikael Sitruk 2012-03-26, 14:21
Juhani hi
By storefile behavior i meant that you look to the metrics and check the number of store file over time and see if the you are bounded or the files increase and decrease all the time. if this is not the case (and the number of store file increase all the time), hbase will throttle the requests. 128-256 bytes each request grouped in 10 is not much data, i have a data set where each request is approx 4K and have insert time of 7-10 ms. Do you see this latency problem on insert during all the test or at some time? Did you check your network latency? BTW batch is not supported by ycsb, so when you mean a set of 10 put you mean the table buffer? in my test it is disabled. Mikael.S On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: > When you increased regions on your previous test, did it start maxing out > CPU? What improvement did you see? > > Have you tried increasing the memstore flush size to something like 512MB? > Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for > a disabled WAL i think, especially with batch size of 10. If you increase > write batch size to 1000 how much does your write throughput increase? > > > On Fri, Mar 23, 2012 at 3:48 AM, Juhani Connolly <[EMAIL PROTECTED]> > wrote: > > > Also, the latency on requests is extremely long. If we group them into > > sets of 10 puts(128-256 bytes each) before flushing the client table, > > latency is over 1 second. > > > > We get entries like this in our logs: > > 22:17:51,010 WARN org.apache.hadoop.ipc.HBaseServer: > > (responseTooSlow): > > > > > {"processingtimems":16692,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@65312e3b > > ), > > rpc version=1, client version=29, > > methodsFingerPrint=54742778","client":"10.172.109.3:42725 > > > ","starttimems":1332335854317,"queuetimems":6387,"class":"HRegionServer","responsesize":0,"method":"multi"} > > > > Any suggestions as to where we should be digging? > > > > On Fri, Mar 23, 2012 at 4:40 PM, Juhani Connolly <[EMAIL PROTECTED]> > > wrote: > > > Status update: > > > > > > - We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were > > > using 0.20.2 series) > > > - Did the tests now with 256/512 regions, the numbers do appear to > > > scale which is good. > > > > > > BUT, our write throughput has gone in the dump. If we disable wal > > > writes, we still get nearly 40,000 a second, but with it on, we're > > > lucky to get more than 12,000. Before we were getting as high as > > > 70,000 grouping puts together. Have set up log collection, and am not > > > finding anything unusual in the logs. > > > > > > Mikael: One of the tests is the ycsb one where we just let it choose > > > the size. Our own custom test has a configurable size, but we have > > > been testing with entries that are 128-256 bytes per entry, as this is > > > what we expect in our application. What exactly should we be looking > > > at with the storefiles? > > > > > > On Wed, Mar 21, 2012 at 2:29 PM, Mikael Sitruk < > [EMAIL PROTECTED]> > > wrote: > > >> Juhani, > > >> Can you look at the storefiles and tell how they behave during the > test? > > >> What is the size of the data you insert/update? > > >> Mikael > > >> On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: > > >> > > >>> Hi Matt, > > >>> > > >>> this is something we haven't tested much, we were always running with > > >>> about 32 regions which gave enough coverage for an even spread over > > >>> all machines. > > >>> I will run our tests with enough regions per server to cover all > cores > > >>> and get back to the ml > > >>> > > >>> On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> > > wrote: > > >>> > I'd be curious to see what happens if you split the table into 1 > > region > > >>> per > > >>> > CPU core, so 24 cores * 11 servers = 264 regions. Each region has > 1 > > >>> > memstore which is a ConcurrentSkipListMap, and you're currently > > hitting > > >>> > each CSLM with 8 cores which might be too contentious. Normally in Mikael.S
-
Re: 0.92 and Read/writes not scalingStack 2012-03-26, 16:29
On Mon, Mar 19, 2012 at 3:41 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote:
> Hi, > > We're running into a brick wall where our throughput numbers will not > scale as we increase server counts both using custom inhouse tests and > ycsb. > Does the above statement still hold? We've moved past the above and we are now on to 'writes are slow'? > We're using hbase 0.92 on hadoop 0.20.2(we also experience the same > issues using 0.90 before switching our testing to this version). > > Our cluster consists of: > - Namenode and hmaster on separate servers, 24 core, 64gb > - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope > to get this changed) > You can put the master and namenode on the same machine. Yes, more disks are better (see the GBIF blog cited in another thread). > - load 10m rows Are the 10m rows for sure spread across all regions? > Delaying WAL flushes gives a small throughput bump but it doesn't > scale. > Why does it not scale? St.Ack
-
Re: 0.92 and Read/writes not scalingStack 2012-03-26, 16:37
On Fri, Mar 23, 2012 at 12:40 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote:
> Status update: > > - We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were > using 0.20.2 series) There is little experience running hbase on hadoop 0.23. > BUT, our write throughput has gone in the dump. If we disable wal > writes, we still get nearly 40,000 a second, but with it on, we're > lucky to get more than 12,000. Before we were getting as high as > 70,000 grouping puts together. Have set up log collection, and am not > finding anything unusual in the logs. > You are batching your writes? Can you paste your configs up on pastebin? And its not a case of high iowait? All else is working properly? No hardware misconfigs or bad switches/nics? (See the yfrog fellows slide deck where they tell a story about all the interesting things they found in their setup apart from hbase configs:http://www.slideshare.net/jacque74/hug-hbase-presentation). St.Ack
-
Re: 0.92 and Read/writes not scalingStack 2012-03-26, 16:42
On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan <[EMAIL PROTECTED]> wrote:
> When you increased regions on your previous test, did it start maxing out > CPU? What improvement did you see? > What Matt asks, what is your cluster doing? What changes do you see when you say, increase size of your batching or as Mat asks, what is the difference when you went from less to more regions? > Have you tried increasing the memstore flush size to something like 512MB? > Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for > a disabled WAL i think, especially with batch size of 10. If you increase > write batch size to 1000 how much does your write throughput increase? > The above sounds like something to try -- upping flush sizes. Are you spending your time compacting all the time? For kicks try disabling compactions when doing your write tests. Does it make a difference? What does ganglia show as hot? Are you network-bound, io-bound, cpu-bound? Thanks, St.Ack
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-26, 16:48
Easiest to answer one mail at a time:
On Mon, Mar 26, 2012 at 10:58 PM, Matt Corgan <[EMAIL PROTECTED]> wrote: > When you increased regions on your previous test, did it start maxing out > CPU? What improvement did you see? > Once we increased regions, to match cpu count, things started scaling, not linearly, but there was definite increases in numbers. > Have you tried increasing the memstore flush size to something like 512MB? > Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for > a disabled WAL i think, especially with batch size of 10. If you increase > write batch size to 1000 how much does your write throughput increase? > We haven't, I'll give it a shot. Things are actually much worse now with hdfs 0.23. But our original scaling problem seems to be sorted by increasing the number of regions to match the number of cpus. The problem now has shifted to extremely slow writes(about 14000/s over 11 servers with a full region count). We may just switch back the hdfs version, but it seems a bit counterproductive :/ > > On Fri, Mar 23, 2012 at 3:48 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > >> Also, the latency on requests is extremely long. If we group them into >> sets of 10 puts(128-256 bytes each) before flushing the client table, >> latency is over 1 second. >> >> We get entries like this in our logs: >> 22:17:51,010 WARN org.apache.hadoop.ipc.HBaseServer: >> (responseTooSlow): >> >> {"processingtimems":16692,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@65312e3b >> ), >> rpc version=1, client version=29, >> methodsFingerPrint=54742778","client":"10.172.109.3:42725 >> ","starttimems":1332335854317,"queuetimems":6387,"class":"HRegionServer","responsesize":0,"method":"multi"} >> >> Any suggestions as to where we should be digging? >> >> On Fri, Mar 23, 2012 at 4:40 PM, Juhani Connolly <[EMAIL PROTECTED]> >> wrote: >> > Status update: >> > >> > - We moved to cdh 4b1, so hbase 0.92 and hdfs 0.23(until now we were >> > using 0.20.2 series) >> > - Did the tests now with 256/512 regions, the numbers do appear to >> > scale which is good. >> > >> > BUT, our write throughput has gone in the dump. If we disable wal >> > writes, we still get nearly 40,000 a second, but with it on, we're >> > lucky to get more than 12,000. Before we were getting as high as >> > 70,000 grouping puts together. Have set up log collection, and am not >> > finding anything unusual in the logs. >> > >> > Mikael: One of the tests is the ycsb one where we just let it choose >> > the size. Our own custom test has a configurable size, but we have >> > been testing with entries that are 128-256 bytes per entry, as this is >> > what we expect in our application. What exactly should we be looking >> > at with the storefiles? >> > >> > On Wed, Mar 21, 2012 at 2:29 PM, Mikael Sitruk <[EMAIL PROTECTED]> >> wrote: >> >> Juhani, >> >> Can you look at the storefiles and tell how they behave during the test? >> >> What is the size of the data you insert/update? >> >> Mikael >> >> On Mar 20, 2012 8:10 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: >> >> >> >>> Hi Matt, >> >>> >> >>> this is something we haven't tested much, we were always running with >> >>> about 32 regions which gave enough coverage for an even spread over >> >>> all machines. >> >>> I will run our tests with enough regions per server to cover all cores >> >>> and get back to the ml >> >>> >> >>> On Tue, Mar 20, 2012 at 1:55 AM, Matt Corgan <[EMAIL PROTECTED]> >> wrote: >> >>> > I'd be curious to see what happens if you split the table into 1 >> region >> >>> per >> >>> > CPU core, so 24 cores * 11 servers = 264 regions. Each region has 1 >> >>> > memstore which is a ConcurrentSkipListMap, and you're currently >> hitting >> >>> > each CSLM with 8 cores which might be too contentious. Normally in >> >>> > production you would want multiple memstores per CPU core. >> >>> > >> >>> > >> >>> > On Mon, Mar 19, 2012 at 5:31 AM, Juhani Connolly <[EMAIL PROTECTED]>
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-26, 16:59
On Mon, Mar 26, 2012 at 11:21 PM, Mikael Sitruk <[EMAIL PROTECTED]> wrote:
> Juhani hi > > By storefile behavior i meant that you look to the metrics and check the > number of store file over time and see if the you are bounded or the files > increase and decrease all the time. if this is not the case (and the number > of store file increase all the time), hbase will throttle the requests. > 128-256 bytes each request grouped in 10 is not much data, i have a data > set where each request is approx 4K and have insert time of 7-10 ms. > Do you see this latency problem on insert during all the test or at some > time? > I'm aware that the writes are small, even batched. So long as there are not too many threads, we were getting very fast writes(3-4ms), though as we increase the thread count, they slow with them(even if the clients are on two separate servers). There is a fixed relation between threadcount and latency, with a seeming hard limit on how many writes will go through(which with hdfs 0.23 is terrible at 14000 over 11 servers). I suspected this might be something along the lines of insufficient connections to the datanodes or something and tried increasing RPC threads on master, namenode, datanode and regionservers, nothing changed this. We have also run the tests on significantly larger pieces of data. Running with 16kb inserts, the throughput drops a little but far from being a linear drop with the payload size... An increase from 256 bytes to 16kb didn't result in more than say a halving of speed(this is from memory, I'll recheck the figures tomorrow from my notes) > Did you check your network latency? Network throughput with iperf is far beyond the throughput that our setup is pushing at the moment. I'll have to check the latency, but on single puts, it is 2-3ms. > BTW batch is not supported by ycsb, so when you mean a set of 10 put you > mean the table buffer? in my test it is disabled. > We did a lot of our tests on ycsb(which incidentally has good write throughput for the load because it disables instant wal flushing), using the run mode the throughput is miserable. I wrote a separate test tool for trying out writes without the wal, multiputs and switching off wal flushing and flushing it manually in the program(and whatever other test took our fancy). Writes that weren't sent to the wall are much much faster(on a table with only one region normal writes were about 3000/s, wal-less writes, 40k/s) I can only guess there is something really weird going on when writing the wal(threads locking each other out?). > Mikael.S > > > On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: > >> When you increased regions on your previous test, did it start maxing out >> CPU? What improvement did you see? >> >> Have you tried increasing the memstore flush size to something like 512MB? >> Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for >> a disabled WAL i think, especially with batch size of 10. If you increase >> write batch size to 1000 how much does your write throughput increase? >> >> >> On Fri, Mar 23, 2012 at 3:48 AM, Juhani Connolly <[EMAIL PROTECTED]> >> wrote: >> >> > Also, the latency on requests is extremely long. If we group them into >> > sets of 10 puts(128-256 bytes each) before flushing the client table, >> > latency is over 1 second. >> > >> > We get entries like this in our logs: >> > 22:17:51,010 WARN org.apache.hadoop.ipc.HBaseServer: >> > (responseTooSlow): >> > >> > >> {"processingtimems":16692,"call":"multi(org.apache.hadoop.hbase.client.MultiAction@65312e3b >> > ), >> > rpc version=1, client version=29, >> > methodsFingerPrint=54742778","client":"10.172.109.3:42725 >> > >> ","starttimems":1332335854317,"queuetimems":6387,"class":"HRegionServer","responsesize":0,"method":"multi"} >> > >> > Any suggestions as to where we should be digging? >> > >> > On Fri, Mar 23, 2012 at 4:40 PM, Juhani Connolly <[EMAIL PROTECTED]> >> > wrote: >> > > Status update:
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-26, 17:02
On Tue, Mar 27, 2012 at 1:29 AM, Stack <[EMAIL PROTECTED]> wrote:
> On Mon, Mar 19, 2012 at 3:41 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: >> Hi, >> >> We're running into a brick wall where our throughput numbers will not >> scale as we increase server counts both using custom inhouse tests and >> ycsb. >> > > Does the above statement still hold? We've moved past the above and > we are now on to 'writes are slow'? > >> We're using hbase 0.92 on hadoop 0.20.2(we also experience the same >> issues using 0.90 before switching our testing to this version). >> >> Our cluster consists of: >> - Namenode and hmaster on separate servers, 24 core, 64gb >> - up to 11 datanode/regionservers. 24 core, 64gb, 4 * 1tb disks(hope >> to get this changed) >> > > You can put the master and namenode on the same machine. > > Yes, more disks are better (see the GBIF blog cited in another thread). > > > >> - load 10m rows > > Are the 10m rows for sure spread across all regions? > > >> Delaying WAL flushes gives a small throughput bump but it doesn't >> scale. >> > > Why does it not scale? > > St.Ack This was on our old setup, things weren't scaling because there weren't enough regions. I had originally meant to make the other thread because the problem was fundamentally different, sorry for the confusion. In summary the problem now is no longer "not scaling"(because as we increase regions to match the available cpu's it seemingly does, just the base numbers are miserable). Instead it is now "since switching to hdfs 0.23 reads are good and scaling but writes are miserably slow(approx 2000 per region)"
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-26, 17:08
On Tue, Mar 27, 2012 at 1:42 AM, Stack <[EMAIL PROTECTED]> wrote:
> On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: >> When you increased regions on your previous test, did it start maxing out >> CPU? What improvement did you see? >> > > What Matt asks, what is your cluster doing? What changes do you see > when you say, increase size of your batching or as Mat asks, what is > the difference when you went from less to more regions? > None of our hardware is even near its limit. Ganglia rarely has a single machine over 25% load, and we have verified io, network, cpu and memory all have plenty of breathing space with other tools(top, iostat, dstat and others mentioned in the hstack article). >> Have you tried increasing the memstore flush size to something like 512MB? >> Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for >> a disabled WAL i think, especially with batch size of 10. If you increase >> write batch size to 1000 how much does your write throughput increase? >> > > The above sounds like something to try -- upping flush sizes. > > Are you spending your time compacting all the time? For kicks try > disabling compactions when doing your write tests. Does it make a > difference? What does ganglia show as hot? Are you network-bound, > io-bound, cpu-bound? > > Thanks, > St.Ack The compaction and flush times according to ganglia are pretty short and insignificant. I've also been watching the rpcs and past events from the html control panel which don't seem to be indicative of a problem. However I will try changing the flushes and using bigger batches, it might turn up something interesting, thanks.
-
Re: 0.92 and Read/writes not scalingTodd Lipcon 2012-03-27, 01:43
Hi Juhani,
I wouldn't have expected CDH4b1 (0.23) to be slower than 0.20 for writes. They should be around the same speed, or even a little faster in some cases. That said, I haven't personally run any benchmarks in several months on this setup. I know our performance/QA team has done some, so I asked them to take a look. Hopefully we should have some results soon. If you can take 10-20 jstacks of the RegionServer and the DN on that same machine while performing your write workload, that would be helpful. It's possible we had a regression during some recent development right before the 4b1 release. If you're feeling adventurous, you can also try upgrading to CDH4b2 snapshot builds, which do have a couple of performance improvements/bugfixes that may help. Drop by #cloudera on IRC and one of us can point you in the right direction if you're willing to try (though of course the nightly builds are somewhat volatile and haven't had any QA) -Todd On Mon, Mar 26, 2012 at 10:08 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > On Tue, Mar 27, 2012 at 1:42 AM, Stack <[EMAIL PROTECTED]> wrote: >> On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan <[EMAIL PROTECTED]> wrote: >>> When you increased regions on your previous test, did it start maxing out >>> CPU? What improvement did you see? >>> >> >> What Matt asks, what is your cluster doing? What changes do you see >> when you say, increase size of your batching or as Mat asks, what is >> the difference when you went from less to more regions? >> > > None of our hardware is even near its limit. Ganglia rarely has a > single machine over 25% load, and we have verified io, network, cpu > and memory all have plenty of breathing space with other tools(top, > iostat, dstat and others mentioned in the hstack article). > >>> Have you tried increasing the memstore flush size to something like 512MB? >>> Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for >>> a disabled WAL i think, especially with batch size of 10. If you increase >>> write batch size to 1000 how much does your write throughput increase? >>> >> >> The above sounds like something to try -- upping flush sizes. >> >> Are you spending your time compacting all the time? For kicks try >> disabling compactions when doing your write tests. Does it make a >> difference? What does ganglia show as hot? Are you network-bound, >> io-bound, cpu-bound? >> >> Thanks, >> St.Ack > > The compaction and flush times according to ganglia are pretty short > and insignificant. I've also been watching the rpcs and past events > from the html control panel which don't seem to be indicative of a > problem. However I will try changing the flushes and using bigger > batches, it might turn up something interesting, thanks. -- Todd Lipcon Software Engineer, Cloudera
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-27, 03:18
Hi Todd,
Here's our thread dumps from one of our slave nodes while running a load. The particular load was set up to grab a table from a tablepool, stop it from autoflushing, put 1000 entries from 128-256 bytes each in(the keys being a random spread throughout the entire keyspace) and then manually flushed. The average latency is an attrocious 58 seconds, though of course it is nothing like that if we use single puts or small batches... Also put in our configs... They had more in them but we stripped them down a lot to try to get at the problem source, no luck though(we took them down to the bare minimum as well but that didn't change things so we restored some of the settings). Thanks, Juhani On 03/27/2012 10:43 AM, Todd Lipcon wrote: > Hi Juhani, > > I wouldn't have expected CDH4b1 (0.23) to be slower than 0.20 for > writes. They should be around the same speed, or even a little faster > in some cases. That said, I haven't personally run any benchmarks in > several months on this setup. I know our performance/QA team has done > some, so I asked them to take a look. Hopefully we should have some > results soon. > > If you can take 10-20 jstacks of the RegionServer and the DN on that > same machine while performing your write workload, that would be > helpful. It's possible we had a regression during some recent > development right before the 4b1 release. If you're feeling > adventurous, you can also try upgrading to CDH4b2 snapshot builds, > which do have a couple of performance improvements/bugfixes that may > help. Drop by #cloudera on IRC and one of us can point you in the > right direction if you're willing to try (though of course the nightly > builds are somewhat volatile and haven't had any QA) > > -Todd > > On Mon, Mar 26, 2012 at 10:08 AM, Juhani Connolly<[EMAIL PROTECTED]> wrote: >> On Tue, Mar 27, 2012 at 1:42 AM, Stack<[EMAIL PROTECTED]> wrote: >>> On Mon, Mar 26, 2012 at 6:58 AM, Matt Corgan<[EMAIL PROTECTED]> wrote: >>>> When you increased regions on your previous test, did it start maxing out >>>> CPU? What improvement did you see? >>>> >>> What Matt asks, what is your cluster doing? What changes do you see >>> when you say, increase size of your batching or as Mat asks, what is >>> the difference when you went from less to more regions? >>> >> None of our hardware is even near its limit. Ganglia rarely has a >> single machine over 25% load, and we have verified io, network, cpu >> and memory all have plenty of breathing space with other tools(top, >> iostat, dstat and others mentioned in the hstack article). >> >>>> Have you tried increasing the memstore flush size to something like 512MB? >>>> Maybe you're blocked on flushes. 40,000 (4,000/server) is pretty slow for >>>> a disabled WAL i think, especially with batch size of 10. If you increase >>>> write batch size to 1000 how much does your write throughput increase? >>>> >>> The above sounds like something to try -- upping flush sizes. >>> >>> Are you spending your time compacting all the time? For kicks try >>> disabling compactions when doing your write tests. Does it make a >>> difference? What does ganglia show as hot? Are you network-bound, >>> io-bound, cpu-bound? >>> >>> Thanks, >>> St.Ack >> The compaction and flush times according to ganglia are pretty short >> and insignificant. I've also been watching the rpcs and past events >> from the html control panel which don't seem to be indicative of a >> problem. However I will try changing the flushes and using bigger >> batches, it might turn up something interesting, thanks. > >
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-28, 08:27
I think there is a lot of stuff in this and the situation has changed a
bit so I'd like to summarize the current situation and verify a few points: Our current environment: - CDH 4b1: hdfs 0.23 and hbase 0.92 - separate master and namenode, 64gb, 24 cores each, colocating with zookeepers(third zookeeper on a separate unshared server) - 11 datanode/regionservers, 24 cores, 64gb, 4 * 1.5tb disks(should become a bottleneck but isn't yet) - Table is split into approx 300 regions and is balanced with from 25-35 regions/server, using snappy compression. Unless otherwise mentioned delayed flushing is disabled The current problem: - Flushed writes seem slow compared to our previous setup(which was the same but using hdfs 0.20.2). - Hardware usage is poor with no visible hardware bottlenecks(this was also the case with our old setup) Evidence: - YCSB, PerformanceEvaluation, application specific throughput test and a generic testing solution(attaching a simplified version that includes the core issues and works standalone) - On our hdfs 0.20.2 setup, we were getting throughput of 40,000 writes/sec(128-256 bytes each), or higher if we delayed log flushes, used batch puts, or similar. - On our new setup, we are getting about 15,000 wps. If we use the non-flushing setup(-t writeunflushed in the attached test) however we can easily push 10 times that - Hardware not creating bottlenecks is generally evidenced by ganglia, top, iostat -d, iperf and a number of others. - We tested append speed with DFSIOTest using 256 byte entries and 10 files, giving us a throughput of 64mb(about 250,000 entries per second in theory then), so wal writes really should be able to keep up with a lot of throughput? One doubt: - While we are fairly confident this is not the case, the only thing I could think of is that there autoFlush was off for our tests with 0.20.2. We used the same test program on both versions, and it is only today that I explicitly set it to off(so it has been working on the default). We never set the writebuffer size. What I'd like to know: - What kind of throughput are people getting on data that is fully AutoFlushed(so every entry is sent to the wall as table.put() is called? Are our figures(a bit over 1000 per node) normal? Or should we be expecting the figures(4-5000 per sec per node) that we were getting on hdfs 0.20.2? - Do people normally see their hardware get anywhere near maxing out on heavy write load? - Is there something wrong with the way we are testing? On 03/27/2012 12:18 PM, Juhani Connolly wrote: > Hi Todd, > > Here's our thread dumps from one of our slave nodes while running a load. > The particular load was set up to grab a table from a tablepool, stop > it from autoflushing, put 1000 entries from 128-256 bytes each in(the > keys being a random spread throughout the entire keyspace) and then > manually flushed. The average latency is an attrocious 58 seconds, > though of course it is nothing like that if we use single puts or > small batches... > Also put in our configs... They had more in them but we stripped them > down a lot to try to get at the problem source, no luck though(we took > them down to the bare minimum as well but that didn't change things > so we restored some of the settings). > > Thanks, > Juhani > > On 03/27/2012 10:43 AM, Todd Lipcon wrote: >> Hi Juhani, >> >> I wouldn't have expected CDH4b1 (0.23) to be slower than 0.20 for >> writes. They should be around the same speed, or even a little faster >> in some cases. That said, I haven't personally run any benchmarks in >> several months on this setup. I know our performance/QA team has done >> some, so I asked them to take a look. Hopefully we should have some >> results soon. >> >> If you can take 10-20 jstacks of the RegionServer and the DN on that >> same machine while performing your write workload, that would be >> helpful. It's possible we had a regression during some recent >> development right before the 4b1 release. If you're feeling
-
RE: 0.92 and Read/writes not scalingBuckley,Ron 2012-03-28, 12:41
Juhani,
We've been working on some similar performance testing on our 50 node cluster running 0.92.1 and CDH3U3. We were looking mostly at reads, but observed similar behavior. HBase wasn't particularly busy, but we couldn't make it go faster. Some debugging later, we found that many (sometimes most) of our responses from HBase would return in 20 or 40 ms. It was kind of interesting to watch, we'd ask for the same row over and over, it would either return in 0 ms, 20 ms, or 40 ms. Looking around we found some related Jira's: https://issues.apache.org/jira/browse/HADOOP-2232 https://issues.apache.org/jira/browse/HADOOP-8069 https://issues.apache.org/jira/browse/HBASE-1177 https://issues.apache.org/jira/browse/HBASE-2125 We added the settings to our below config to disable Nagle. For us, setting these two, got rid of all of the 20 and 40 ms response times and dropped the average response time we measured from HBase by more than half. Plus, we can push HBase a lot harder. hbase-site.xml <property> <name>hbase.ipc.client.tcpnodelay</name> <value>true</value> </property> <property> <name>ipc.server.tcpnodelay</name> <value>true</value> </property> core-site.xml <property> <name>ipc.server.tcpnodelay</name> <value>true</value> </property> -----Original Message----- From: Juhani Connolly [mailto:[EMAIL PROTECTED]] Sent: Wednesday, March 28, 2012 4:27 AM To: [EMAIL PROTECTED] Subject: Re: 0.92 and Read/writes not scaling I think there is a lot of stuff in this and the situation has changed a bit so I'd like to summarize the current situation and verify a few points: Our current environment: - CDH 4b1: hdfs 0.23 and hbase 0.92 - separate master and namenode, 64gb, 24 cores each, colocating with zookeepers(third zookeeper on a separate unshared server) - 11 datanode/regionservers, 24 cores, 64gb, 4 * 1.5tb disks(should become a bottleneck but isn't yet) - Table is split into approx 300 regions and is balanced with from 25-35 regions/server, using snappy compression. Unless otherwise mentioned delayed flushing is disabled The current problem: - Flushed writes seem slow compared to our previous setup(which was the same but using hdfs 0.20.2). - Hardware usage is poor with no visible hardware bottlenecks(this was also the case with our old setup) Evidence: - YCSB, PerformanceEvaluation, application specific throughput test and a generic testing solution(attaching a simplified version that includes the core issues and works standalone) - On our hdfs 0.20.2 setup, we were getting throughput of 40,000 writes/sec(128-256 bytes each), or higher if we delayed log flushes, used batch puts, or similar. - On our new setup, we are getting about 15,000 wps. If we use the non-flushing setup(-t writeunflushed in the attached test) however we can easily push 10 times that - Hardware not creating bottlenecks is generally evidenced by ganglia, top, iostat -d, iperf and a number of others. - We tested append speed with DFSIOTest using 256 byte entries and 10 files, giving us a throughput of 64mb(about 250,000 entries per second in theory then), so wal writes really should be able to keep up with a lot of throughput? One doubt: - While we are fairly confident this is not the case, the only thing I could think of is that there autoFlush was off for our tests with 0.20.2. We used the same test program on both versions, and it is only today that I explicitly set it to off(so it has been working on the default). We never set the writebuffer size. What I'd like to know: - What kind of throughput are people getting on data that is fully AutoFlushed(so every entry is sent to the wall as table.put() is called? Are our figures(a bit over 1000 per node) normal? Or should we be expecting the figures(4-5000 per sec per node) that we were getting on hdfs 0.20.2? - Do people normally see their hardware get anywhere near maxing out on heavy write load? - Is there something wrong with the way we are testing? On 03/27/2012 12:18 PM, Juhani Connolly wrote: load. nightly see is
-
Re: 0.92 and Read/writes not scalingStack 2012-03-28, 17:12
On Wed, Mar 28, 2012 at 5:41 AM, Buckley,Ron <[EMAIL PROTECTED]> wrote:
> For us, setting these two, got rid of all of the 20 and 40 ms response > times and dropped the average response time we measured from HBase by > more than half. Plus, we can push HBase a lot harder. > That had an effect on random read workload only Ron? Thanks, St.Ack
-
RE: 0.92 and Read/writes not scalingBuckley,Ron 2012-03-28, 17:56
Stack,
We're about 80% random read and 20% random write. So, that would have been the mix that we were running. We'll try a test with Nagel On and then Nagel off, random write only, later this afternoon and see if the same pattern emerges. Ron -----Original Message----- From: [EMAIL PROTECTED] [mailto:[EMAIL PROTECTED]] On Behalf Of Stack Sent: Wednesday, March 28, 2012 1:12 PM To: [EMAIL PROTECTED] Subject: Re: 0.92 and Read/writes not scaling On Wed, Mar 28, 2012 at 5:41 AM, Buckley,Ron <[EMAIL PROTECTED]> wrote: > For us, setting these two, got rid of all of the 20 and 40 ms response > times and dropped the average response time we measured from HBase by > more than half. Plus, we can push HBase a lot harder. > That had an effect on random read workload only Ron? Thanks, St.Ack
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-03-29, 02:36
Ron,
thanks for sharing those settings. Unfortunately they didn't help with our read throughput, but every little bit helps. Another suspicious thing that has come up is with the network... While overall throughput has been verified to be able to go much higher than the tax hbase is putting on it right now, there seem to be errors and dropped packets(though this is relative to a massive amount of traffic): [juhani_connolly@hornet-slave01 ~]$ sudo /sbin/ifconfig bond0 パスワード: bond0 Link encap:Ethernet HWaddr 78:2B:CB:59:A9:34 inet addr:******** Bcast:********** Mask:255.255.0.0 inet6 addr: fe80::7a2b:cbff:fe59:a934/64 Scope:Link UP BROADCAST RUNNING MASTER MULTICAST MTU:1500 Metric:1 RX packets:9422705447 errors:605 dropped:6222 overruns:0 frame:605 TX packets:9317689449 errors:0 dropped:0 overruns:0 carrier:0 collisions:0 txqueuelen:0 RX bytes:6609813756075 (6.0 TiB) TX bytes:6033761947482 (5.4 TiB) could this possibly be a problem cause? Since we haven't heard anything on expected throughput we're downgrading our hdfs back to 0.20.2, I'd be curious to hear how other people do with 0.23 and the throughput they're getting. On 03/29/2012 02:56 AM, Buckley,Ron wrote: > Stack, > > We're about 80% random read and 20% random write. So, that would have been the mix that we were running. > > We'll try a test with Nagel On and then Nagel off, random write only, later this afternoon and see if the same pattern emerges. > > Ron > > -----Original Message----- > From: [EMAIL PROTECTED] [mailto:[EMAIL PROTECTED]] On Behalf Of Stack > Sent: Wednesday, March 28, 2012 1:12 PM > To: [EMAIL PROTECTED] > Subject: Re: 0.92 and Read/writes not scaling > > On Wed, Mar 28, 2012 at 5:41 AM, Buckley,Ron<[EMAIL PROTECTED]> wrote: >> For us, setting these two, got rid of all of the 20 and 40 ms response >> times and dropped the average response time we measured from HBase by >> more than half. Plus, we can push HBase a lot harder. >> > That had an effect on random read workload only Ron? > Thanks, > St.Ack > > >
-
Re: 0.92 and Read/writes not scalingAlok Singh 2012-04-02, 17:15
Sorry for jumping on this thread late, but, I have seen very similar
behavior in our cluster with hadoop 0.23.2 (CDH4B2 snapshot) and hbase 0.23.1. We have a small, 7 node cluster (48GB/16Core/6x10Kdisk/GigE network) with about 500M rows/4Tb of data. The random read performance is excellent, but, random write throughput maxes out around 10K/sec. Turning off the WAL takes it up to 40-50k/sec, but, that's not something we will leave off in production. One of the settings that I experimented with was hbase.hregion.max.filesize. Increasing it to 10GB actually made the write throughput worse, so, I have set it back down to 2GB. Later this week, I will attempt to do another cycle of tests and hopefully have some thread dumps to report back with. Alok 2012/3/30 Doug Meil <[EMAIL PROTECTED]>: > > Just as a quick reminder regarding what Todd mentioned, that's exactly > what was happening in this case study... > > http://hbase.apache.org/book.html#casestudies.slownode > > ... although it doesn't appear to be the problem in this particular > situation. > > > > > On 3/29/12 8:22 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: > >>On Fri, Mar 30, 2012 at 7:36 AM, Todd Lipcon <[EMAIL PROTECTED]> wrote: >>> On the other hand, I've seen that "frame errors" are often correlated >>> with NICs auto-negotiating to the wrong speed, etc. Double check with >>> ethtool that all of your machines are gigabit full-duplex and not >>> doing something strange. Also double check your bonding settings, etc. >>> >>> -Todd >>> >> >>I did this after seeing the errors on ifconfig, but everything looks >>ok on that front: >>Settings for eth0: >> Supported ports: [ TP ] >> Supported link modes: 10baseT/Half 10baseT/Full >> 100baseT/Half 100baseT/Full >> 1000baseT/Full >> Supports auto-negotiation: Yes >> Advertised link modes: 10baseT/Half 10baseT/Full >> 100baseT/Half 100baseT/Full >> 1000baseT/Full >> Advertised auto-negotiation: Yes >> Speed: 1000Mb/s >> Duplex: Full >> Port: Twisted Pair >> PHYAD: 1 >> Transceiver: internal >> Auto-negotiation: on >> Supports Wake-on: g >> Wake-on: d >> Link detected: yes >> >>Also, since yesterday the error counts have not increased at all so I >>guess that was just a red herring... >> >> >>> 2012/3/28 Dave Wang <[EMAIL PROTECTED]>: >>>> As you said, the amount of errors and drops you are seeing are very >>>>small >>>> compared to your overall traffic, so I doubt that is a significant >>>> contributor to the throughput problems you are seeing. >>>> >>>> - Dave >>>> >>>> On Wed, Mar 28, 2012 at 7:36 PM, Juhani Connolly < >>>> [EMAIL PROTECTED]> wrote: >>>> >>>>> Ron, >>>>> >>>>> thanks for sharing those settings. Unfortunately they didn't help >>>>>with our >>>>> read throughput, but every little bit helps. >>>>> >>>>> Another suspicious thing that has come up is with the network... While >>>>> overall throughput has been verified to be able to go much higher >>>>>than the >>>>> tax hbase is putting on it right now, there seem to be errors and >>>>>dropped >>>>> packets(though this is relative to a massive amount of traffic): >>>>> >>>>> [juhani_connolly@hornet-**slave01 ~]$ sudo /sbin/ifconfig bond0 >>>>> パスワ�ド: >>>>> bond0 Link encap:Ethernet HWaddr 78:2B:CB:59:A9:34 >>>>> inet addr:******** Bcast:********** Mask:255.255.0.0 >>>>> inet6 addr: fe80::7a2b:cbff:fe59:a934/64 Scope:Link >>>>> UP BROADCAST RUNNING MASTER MULTICAST MTU:1500 Metric:1 >>>>> RX packets:9422705447 errors:605 dropped:6222 overruns:0 frame:605 >>>>> TX packets:9317689449 errors:0 dropped:0 overruns:0 carrier:0 >>>>> collisions:0 txqueuelen:0 >>>>> RX bytes:6609813756075 (6.0 TiB) TX bytes:6033761947482 (5.4 TiB) >>>>> >>>>> could this possibly be a problem cause? >>>>> Since we haven't heard anything on expected throughput we're
-
Re: 0.92 and Read/writes not scalingStack 2012-04-02, 17:41
2012/4/2 Alok Singh <[EMAIL PROTECTED]>:
> Sorry for jumping on this thread late, but, I have seen very similar > behavior in our cluster with hadoop 0.23.2 (CDH4B2 snapshot) and hbase > 0.23.1. We have a small, 7 node cluster (48GB/16Core/6x10Kdisk/GigE > network) with about 500M rows/4Tb of data. The random read performance > is excellent, but, random write throughput maxes out around 10K/sec. > Turning off the WAL takes it up to 40-50k/sec, but, that's not > something we will leave off in production. > > One of the settings that I experimented with was > hbase.hregion.max.filesize. Increasing it to 10GB actually made the > write throughput worse, so, I have set it back down to 2GB. Later this > week, I will attempt to do another cycle of tests and hopefully have > some thread dumps to report back with. > Thanks for writing the list Alok. Juhani is going to come back on this thread saying he went back to 0.20.x hadoop to get his write performance back. I'll let him respond. Seems like we have an issue w/ 0.23 hadoop WAL writes going by yours and Juhani's experience. St.Ack
-
Re: 0.92 and Read/writes not scalingJonathan Hsieh 2012-04-03, 00:21
Juhani,
Have you looked at any of the logs from your perf runs? Can you try running HBase's performance evaluation with debug comments on? I'd like to know if what I'm seeing is the same as you. I've started running some of these and have encountered what seems to be networking code isssues (SocketTimeoutExceptions, a bunch of delayedAcks in ganglia, and 4x-5x degradation in write's from 0.90 runs to 0.92 runs). == cmd lines: hbase org.apache.hadoop.hbase.PerformanceEvaluation sequentialWrite 1 hbase org.apache.hadoop.hbase.PerformanceEvaluation randomWrite 1 == in log4j.properties log4j.logger.org.apache.hadoop.hbase=DEBUG Jon. On Thu, Mar 29, 2012 at 12:05 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > On Thu, Mar 29, 2012 at 1:10 PM, Stack <[EMAIL PROTECTED]> wrote: > > On Wed, Mar 28, 2012 at 7:36 PM, Juhani Connolly > > <[EMAIL PROTECTED]> wrote: > >> Since we haven't heard anything on expected throughput we're > downgrading our > >> hdfs back to 0.20.2, I'd be curious to hear how other people do with > 0.23 > >> and the throughput they're getting. > >> > > > > We don't have much experience running on 0.23, I think its fair to > > say. It works but not much more than that can be said. The sync code > > path is different in 0.23 than in 0.20.2 and has had less scrutiny > > (When you say 0.20.2, you mean CDH? Which CDH?). I think its good to > > go back. > > Thanks for the info on 0.23. I suspect that the change in sync you > mentioned may well have something to do with this, since decreasing > the frequency of appends through the use of a moderate sized > writeBuffer at the client end pays huge dividends(as of course does > removing the appends altogether by disabling wal writes). High counts > of ungrouped(whether that be by group puts, delayed client flushing or > delayed WAL flushing) writes seem to suffer pretty badly under 0.23. > We'll be moving back to 0.20.2 as it seems to be much better tested > and stressed, likely to the cdh distro(3u3). > > > > > Regards numbers, its hard to compare workloads but if it helps, > > looking at our frontend now, its relatively idle doing between > > 100-500k hits on 30 machines that are less than yours, less memory, > > 10k regions, with a workload that is mostly increments > > (read-mostly-from-block-cache/modify/write). > > > > Thanks... It's nice to have a frame of reference to compare against. > > > Yes, the errors are relatively few but poke around more if you can. > > Why are there errors at all? > > St.Ack > > I'm not sure. As has been said, likely unrelated, going to try and > figure it out. > > Thanks, > Juhani > -- // Jonathan Hsieh (shay) // Software Engineer, Cloudera // [EMAIL PROTECTED]
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-04-03, 02:52
Hi guys, conversation went off the list briefly as I resent stack
dumps to stack. We've moved back to hdfs 0.20.2 but want to post this back here and try to summarize events as well as our experiences with 0.23 and concerns. Quick summary: after having some issues with 0.20.2(since resolved), we tried moving to full cdh4b1(which consists of hadoop 0.23 and hbase 0.92.0) Upon starting running tests on hadoop 0.23, while we found read performance to be great, we found that write autoFlushed throughput was pretty bad(about 1000/wps... as a point of reference, with 0.20.2 we are now getting 10k/wps with autoFlush on). We used multiple different tests: Hbases PerformanceEvaluation class, YCSB and our own custom test which allowed us to compare throughput numbers for increments, reads, unbuffered/buffered writes, etc What we found from these tests is that if we flushed writes immediately(for example by having autoFlush set to true), our throughput was poor. However if we disabled the wall, or just used a moderate sized writeBuffer(default of 2mb), disabling autoFlush(resulting in appends of roughly 200kb of data per server, of which there were 11), that we got good throughput. The numbers we got for 0.20.2 and 0.23 are using the same settings and same tests. Once we moved back to 0.20.2 we found that our test client would become the bottleneck for some tests and had to run two simultaneously to get the cluster to work up any sweat at all. With 0.23 adding further clients only increased the latency on individual puts, leading us to believe there was something wrong with appends. Stack suggested that the changes in Sync behavior in 0.23 may have something to do with it, I have been unable to confirm this as I am not very familiar with the hdfs codebase. I include the backlog(edited for clarity) of my conversation with stack, also with some inline comments, hopefully this can be of help to people trying to figure out how to debug stuff: Most of this was over the weekend when I couldn't get at the servers, added in details where I can Stack takes a look at the thread dumps: >>>>>>> The servers seem to be doing nothing. In the last thread dump, >>>>>>> 2012-03-30 08:48:09, all is idle in essence but for one thread doing >>>>>>> this: >>>>>>> "IPC Server handler 14 on 60020" daemon prio=10 tid=0x00002aaac80f0000 >>>>>>> nid=0x2eae runnable [0x0000000046259000] >>>>>>> java.lang.Thread.State: RUNNABLE >>>>>>> at org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1499) >>>>>>> - locked <0x00000004038d1cd8> (a org.apache.hadoop.hdfs.DFSOutputStream) >>>>>>> at org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1417) >>>>>>> - locked <0x00000004038d1cd8> (a org.apache.hadoop.hdfs.DFSOutputStream) >>>>>>> at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116) >>>>>>> at org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:1199) >>>>>>> at sun.reflect.GeneratedMethodAccessor13.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.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:228) >>>>>>> at org.apache.hadoop.hbase.regionserver.wal.HLog.syncer(HLog.java:1084) >>>>>>> at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1195) >>>>>>> at org.apache.hadoop.hbase.regionserver.wal.HLog.append(HLog.java:1032) >>>>>>> at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:1853) >>>>>>> at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1723) >>>>>>> at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3062) >>>>>>> at sun.reflect.GeneratedMethodAccessor39.invoke(Unknown Source) >>>>>>> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) Essentially since we only have one rpc connection per region server(see the implementation of HConnection/HConnectionManager) going out from the client, that only results in a single thread from each region server working on hbase. We did try more clients but interestingly enough this did not result in better throughput, just the same throughput with a higher latency per request(since there were more threads trying to share the same throughput) This single connection per rs behavior can also be changed with the following configuration settings: hbase.client.ipc.pool.size , hbase.client.ipc.pool.type Our memory settings should have been solid and aren't causing any weirdness on 0.20.2 now... I have no idea what was going on here. Again due to shortage of clients, but even with more clients throughput doesn't go up. The block cache was enabled, set to 35%. The tables in the test were set to IN_MEMORY=>TRUE. autoFlush is accessible from HTable.setAutoFlush, if it's set to false, puts are stored at client size until either manually flushed or until the writeBuffer is full, when they will be flushed. By storing up a moderate number of items they can be batched together before sending to the targetted regionserver(this is all done automatically in multiPut which divvies up the writeBuffer by regionServer, then sends the blocks to each server resulting in larger appends, and less small packages). By disabling autoFlushing however, there will be a delay of data being stored(until the writeBuffer is full), and if the client fails before a flush, puts can be lost. Finally as an aside, reads, increments and deletes do not go in the writeBuffer and are processed immediately. Since moving to 0.20.2 we are finally seeing our region servers push up a sweat. No matter what we did before they would never be taxed much. Disabling the wal will increase throughput significantly but the throughput is so
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-04-03, 02:56
Hi Alok, please refer to my previous post where I detailed some of the
stuff we did. At this point, I'm unsure if it is actually possible to get good autoFlushed throughput with 0.23, we weren't able to and switched back to 0.20.2 If you want to persevere however, please let us know if you make any breakthroughs! 2012/4/3 Alok Singh <[EMAIL PROTECTED]>: > Sorry for jumping on this thread late, but, I have seen very similar > behavior in our cluster with hadoop 0.23.2 (CDH4B2 snapshot) and hbase > 0.23.1. We have a small, 7 node cluster (48GB/16Core/6x10Kdisk/GigE > network) with about 500M rows/4Tb of data. The random read performance > is excellent, but, random write throughput maxes out around 10K/sec. > Turning off the WAL takes it up to 40-50k/sec, but, that's not > something we will leave off in production. > > One of the settings that I experimented with was > hbase.hregion.max.filesize. Increasing it to 10GB actually made the > write throughput worse, so, I have set it back down to 2GB. Later this > week, I will attempt to do another cycle of tests and hopefully have > some thread dumps to report back with. > > Alok > > > 2012/3/30 Doug Meil <[EMAIL PROTECTED]>: >> >> Just as a quick reminder regarding what Todd mentioned, that's exactly >> what was happening in this case study... >> >> http://hbase.apache.org/book.html#casestudies.slownode >> >> ... although it doesn't appear to be the problem in this particular >> situation. >> >> >> >> >> On 3/29/12 8:22 PM, "Juhani Connolly" <[EMAIL PROTECTED]> wrote: >> >>>On Fri, Mar 30, 2012 at 7:36 AM, Todd Lipcon <[EMAIL PROTECTED]> wrote: >>>> On the other hand, I've seen that "frame errors" are often correlated >>>> with NICs auto-negotiating to the wrong speed, etc. Double check with >>>> ethtool that all of your machines are gigabit full-duplex and not >>>> doing something strange. Also double check your bonding settings, etc. >>>> >>>> -Todd >>>> >>> >>>I did this after seeing the errors on ifconfig, but everything looks >>>ok on that front: >>>Settings for eth0: >>> Supported ports: [ TP ] >>> Supported link modes: 10baseT/Half 10baseT/Full >>> 100baseT/Half 100baseT/Full >>> 1000baseT/Full >>> Supports auto-negotiation: Yes >>> Advertised link modes: 10baseT/Half 10baseT/Full >>> 100baseT/Half 100baseT/Full >>> 1000baseT/Full >>> Advertised auto-negotiation: Yes >>> Speed: 1000Mb/s >>> Duplex: Full >>> Port: Twisted Pair >>> PHYAD: 1 >>> Transceiver: internal >>> Auto-negotiation: on >>> Supports Wake-on: g >>> Wake-on: d >>> Link detected: yes >>> >>>Also, since yesterday the error counts have not increased at all so I >>>guess that was just a red herring... >>> >>> >>>> 2012/3/28 Dave Wang <[EMAIL PROTECTED]>: >>>>> As you said, the amount of errors and drops you are seeing are very >>>>>small >>>>> compared to your overall traffic, so I doubt that is a significant >>>>> contributor to the throughput problems you are seeing. >>>>> >>>>> - Dave >>>>> >>>>> On Wed, Mar 28, 2012 at 7:36 PM, Juhani Connolly < >>>>> [EMAIL PROTECTED]> wrote: >>>>> >>>>>> Ron, >>>>>> >>>>>> thanks for sharing those settings. Unfortunately they didn't help >>>>>>with our >>>>>> read throughput, but every little bit helps. >>>>>> >>>>>> Another suspicious thing that has come up is with the network... While >>>>>> overall throughput has been verified to be able to go much higher >>>>>>than the >>>>>> tax hbase is putting on it right now, there seem to be errors and >>>>>>dropped >>>>>> packets(though this is relative to a massive amount of traffic): >>>>>> >>>>>> [juhani_connolly@hornet-**slave01 ~]$ sudo /sbin/ifconfig bond0 >>>>>> パスワ�ド: >>>>>> bond0 Link encap:Ethernet HWaddr 78:2B:CB:59:A9:34 >>>>>> inet addr:******** Bcast:********** Mask:255.255.0.0 >>>
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-04-03, 03:02
Jon,
we had a fair few long pauses. Our test tool gave us latency, and we got a lot of requests taking much longer than they should. Unfortunately we didn't hold onto our logs from the PerformanceEvaluation runs. Also I would note that PerformanceEvaluation internally disables autoFlush, so it does not run into the issues I have described. I would recommend running some code that has autoWrite set to true to test this problem. We've moved our environment back to 0.20.2 as we start testing before using it in production, so unfortunately we can't run any more tests on it, sorry :/ On Tue, Apr 3, 2012 at 9:21 AM, Jonathan Hsieh <[EMAIL PROTECTED]> wrote: > Juhani, > > Have you looked at any of the logs from your perf runs? Can you try > running HBase's performance evaluation with debug comments on? I'd like > to know if what I'm seeing is the same as you. > > I've started running some of these and have encountered what seems to be > networking code isssues (SocketTimeoutExceptions, a bunch of delayedAcks in > ganglia, and 4x-5x degradation in write's from 0.90 runs to 0.92 runs). > > == cmd lines: > hbase org.apache.hadoop.hbase.PerformanceEvaluation sequentialWrite 1 > hbase org.apache.hadoop.hbase.PerformanceEvaluation randomWrite 1 > > == in log4j.properties > log4j.logger.org.apache.hadoop.hbase=DEBUG > > > Jon. > > > On Thu, Mar 29, 2012 at 12:05 AM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > >> On Thu, Mar 29, 2012 at 1:10 PM, Stack <[EMAIL PROTECTED]> wrote: >> > On Wed, Mar 28, 2012 at 7:36 PM, Juhani Connolly >> > <[EMAIL PROTECTED]> wrote: >> >> Since we haven't heard anything on expected throughput we're >> downgrading our >> >> hdfs back to 0.20.2, I'd be curious to hear how other people do with >> 0.23 >> >> and the throughput they're getting. >> >> >> > >> > We don't have much experience running on 0.23, I think its fair to >> > say. It works but not much more than that can be said. The sync code >> > path is different in 0.23 than in 0.20.2 and has had less scrutiny >> > (When you say 0.20.2, you mean CDH? Which CDH?). I think its good to >> > go back. >> >> Thanks for the info on 0.23. I suspect that the change in sync you >> mentioned may well have something to do with this, since decreasing >> the frequency of appends through the use of a moderate sized >> writeBuffer at the client end pays huge dividends(as of course does >> removing the appends altogether by disabling wal writes). High counts >> of ungrouped(whether that be by group puts, delayed client flushing or >> delayed WAL flushing) writes seem to suffer pretty badly under 0.23. >> We'll be moving back to 0.20.2 as it seems to be much better tested >> and stressed, likely to the cdh distro(3u3). >> >> > >> > Regards numbers, its hard to compare workloads but if it helps, >> > looking at our frontend now, its relatively idle doing between >> > 100-500k hits on 30 machines that are less than yours, less memory, >> > 10k regions, with a workload that is mostly increments >> > (read-mostly-from-block-cache/modify/write). >> > >> >> Thanks... It's nice to have a frame of reference to compare against. >> >> > Yes, the errors are relatively few but poke around more if you can. >> > Why are there errors at all? >> > St.Ack >> >> I'm not sure. As has been said, likely unrelated, going to try and >> figure it out. >> >> Thanks, >> Juhani >> > > > > -- > // Jonathan Hsieh (shay) > // Software Engineer, Cloudera > // [EMAIL PROTECTED]
-
Re: 0.92 and Read/writes not scalingJonathan Hsieh 2012-04-03, 03:19
The interesting point I didn't mention from my simplistic tests is that
these slowdowns were present when using 0.92ish hbase on top of cdh3u3 hdfs (the olds school hadoop 0.20.x based hadoop and didn't even use a hadoop 23 based hdfs) . I'm in the process of testing a hypothesis Todd suggested and will share results after test is done. Jon. On Mon, Apr 2, 2012 at 8:02 PM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > Jon, > > we had a fair few long pauses. Our test tool gave us latency, and we > got a lot of requests taking much longer than they should. > Unfortunately we didn't hold onto our logs from the PerformanceEvaluation > runs. > > Also I would note that PerformanceEvaluation internally disables > autoFlush, so it does not run into the issues I have described. I > would recommend running some code that has autoWrite set to true to > test this problem. > > We've moved our environment back to 0.20.2 as we start testing before > using it in production, so unfortunately we can't run any more tests > on it, sorry :/ > > On Tue, Apr 3, 2012 at 9:21 AM, Jonathan Hsieh <[EMAIL PROTECTED]> wrote: > > Juhani, > > > > Have you looked at any of the logs from your perf runs? Can you try > > running HBase's performance evaluation with debug comments on? I'd like > > to know if what I'm seeing is the same as you. > > > > I've started running some of these and have encountered what seems to be > > networking code isssues (SocketTimeoutExceptions, a bunch of delayedAcks > in > > ganglia, and 4x-5x degradation in write's from 0.90 runs to 0.92 runs). > > > > == cmd lines: > > hbase org.apache.hadoop.hbase.PerformanceEvaluation sequentialWrite 1 > > hbase org.apache.hadoop.hbase.PerformanceEvaluation randomWrite 1 > > > > == in log4j.properties > > log4j.logger.org.apache.hadoop.hbase=DEBUG > > > > > > Jon. > > > > > > On Thu, Mar 29, 2012 at 12:05 AM, Juhani Connolly <[EMAIL PROTECTED]> > wrote: > > > >> On Thu, Mar 29, 2012 at 1:10 PM, Stack <[EMAIL PROTECTED]> wrote: > >> > On Wed, Mar 28, 2012 at 7:36 PM, Juhani Connolly > >> > <[EMAIL PROTECTED]> wrote: > >> >> Since we haven't heard anything on expected throughput we're > >> downgrading our > >> >> hdfs back to 0.20.2, I'd be curious to hear how other people do with > >> 0.23 > >> >> and the throughput they're getting. > >> >> > >> > > >> > We don't have much experience running on 0.23, I think its fair to > >> > say. It works but not much more than that can be said. The sync code > >> > path is different in 0.23 than in 0.20.2 and has had less scrutiny > >> > (When you say 0.20.2, you mean CDH? Which CDH?). I think its good to > >> > go back. > >> > >> Thanks for the info on 0.23. I suspect that the change in sync you > >> mentioned may well have something to do with this, since decreasing > >> the frequency of appends through the use of a moderate sized > >> writeBuffer at the client end pays huge dividends(as of course does > >> removing the appends altogether by disabling wal writes). High counts > >> of ungrouped(whether that be by group puts, delayed client flushing or > >> delayed WAL flushing) writes seem to suffer pretty badly under 0.23. > >> We'll be moving back to 0.20.2 as it seems to be much better tested > >> and stressed, likely to the cdh distro(3u3). > >> > >> > > >> > Regards numbers, its hard to compare workloads but if it helps, > >> > looking at our frontend now, its relatively idle doing between > >> > 100-500k hits on 30 machines that are less than yours, less memory, > >> > 10k regions, with a workload that is mostly increments > >> > (read-mostly-from-block-cache/modify/write). > >> > > >> > >> Thanks... It's nice to have a frame of reference to compare against. > >> > >> > Yes, the errors are relatively few but poke around more if you can. > >> > Why are there errors at all? > >> > St.Ack > >> > >> I'm not sure. As has been said, likely unrelated, going to try and > >> figure it out. > >> > >> Thanks, > >> Juhani > >> > > > > > > // Jonathan Hsieh (shay) // Software Engineer, Cloudera // [EMAIL PROTECTED]
-
Re: 0.92 and Read/writes not scalingStack 2012-04-03, 03:50
On Mon, Apr 2, 2012 at 8:19 PM, Jonathan Hsieh <[EMAIL PROTECTED]> wrote:
> I'm in the process of testing a hypothesis Todd suggested > and will share results after test is done. > What is the hypothesis? St.Ack
-
Re: 0.92 and Read/writes not scalingJonathan Hsieh 2012-04-03, 16:56
The hypothesis was that since I was seeing TCP ack delays in ganglia, it
may have to do with the TCP_NODELAY setting on the write side. The hdfs client sets this in the read side DFSInputStream, here but not on the DFSOutputStream write side: https://github.com/apache/hadoop-common/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L836 // TCP_NODELAY is crucial here because of bad interactions between // Nagle's Algorithm and Delayed ACKs. With connection keepalive // between the client and DN, the conversation looks like: // 1. Client -> DN: Read block X // 2. DN -> Client: data for block X // 3. Client -> DN: Status OK (successful read) // 4. Client -> DN: Read block Y // The fact that step #3 and #4 are both in the client->DN direction // triggers Nagling. If the DN is using delayed ACKs, this results // in a delay of 40ms or more. // The fact that I am getting ackDelays on a write test may indicate that we need this set TCP_NODELAY on the HBase HLog write side -- (HDFS's DFSClient.DFSOutputStream in hadoop 0.20.x and DFSOutputStream in 0.23.) I did a quick hack and test adding socket.setNoTcpDelay(true) on that write side of a hadoop 0.20.x and reran the PE tests; unfortunately, we still seem to have the socketTimeoutException problems. Needs more digging.. Jon On Mon, Apr 2, 2012 at 8:50 PM, Stack <[EMAIL PROTECTED]> wrote: > On Mon, Apr 2, 2012 at 8:19 PM, Jonathan Hsieh <[EMAIL PROTECTED]> wrote: > > I'm in the process of testing a hypothesis Todd suggested > > and will share results after test is done. > > > > What is the hypothesis? > St.Ack > -- // Jonathan Hsieh (shay) // Software Engineer, Cloudera // [EMAIL PROTECTED]
-
Re: 0.92 and Read/writes not scalingStack 2012-04-03, 17:42
On Tue, Apr 3, 2012 at 9:56 AM, Jonathan Hsieh <[EMAIL PROTECTED]> wrote:
> The hypothesis was that since I was seeing TCP ack delays in ganglia, it > may have to do with the TCP_NODELAY setting on the write side. The hdfs > client sets this in the read side DFSInputStream, here but not on the > DFSOutputStream write side: > > https://github.com/apache/hadoop-common/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L836 > > > // TCP_NODELAY is crucial here because of bad interactions between > // Nagle's Algorithm and Delayed ACKs. With connection keepalive > // between the client and DN, the conversation looks like: > // 1. Client -> DN: Read block X > // 2. DN -> Client: data for block X > // 3. Client -> DN: Status OK (successful read) > // 4. Client -> DN: Read block Y > // The fact that step #3 and #4 are both in the client->DN direction > // triggers Nagling. If the DN is using delayed ACKs, this results > // in a delay of 40ms or more. > // > > The fact that I am getting ackDelays on a write test may indicate that we > need this set TCP_NODELAY on the HBase HLog write side -- > (HDFS's DFSClient.DFSOutputStream in hadoop 0.20.x and DFSOutputStream in > 0.23.) I did a quick hack and test adding socket.setNoTcpDelay(true) on > that write side of a hadoop 0.20.x and reran the PE tests; unfortunately, > we still seem to have the socketTimeoutException problems. Needs more > digging.. > Thanks Jon for the above. St.Ack
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-04-05, 01:45
another quick update on stuff:
since moving back to hdfs 0.20.2 (with hbase still at 0.92), we found that while we made significant gains in throughput, that most of our regionservers IPC threads were stuck somewhere in HWal.append(out of 50, 42 were in append, of which 20 were in sync), limiting throughput despite significant free hardware resources. Because the WAL writes of a single RS all go sequentially to one HDFS file, we assumed that we could improve throughput by separating writes to more WAL files and more HDs. To do this we ran multiple region servers on each node. The scaling wasn't linear(we were in no way increasing hardware, just the number of regionservers), but we are now getting significantly more throughput. I would personally not say that this is a great approach to have to take, it would generally be better to build more smaller servers which will thus not limit themselves by trying to put a lot of data per server through a single WAL file. Of course there may be another solution to this that I'm not aware of? If so I'd love to hear it.
-
Re: 0.92 and Read/writes not scalingTed Yu 2012-04-05, 02:43
Juhani:
Thanks for sharing your results. Do you mind putting the summary on HBASE-5699: Run with > 1 WAL in HRegionServer ? On Wed, Apr 4, 2012 at 6:45 PM, Juhani Connolly < [EMAIL PROTECTED]> wrote: > another quick update on stuff: > > since moving back to hdfs 0.20.2 (with hbase still at 0.92), we found that > while we made significant gains in throughput, that most of our > regionservers IPC threads were stuck somewhere in HWal.append(out of 50, 42 > were in append, of which 20 were in sync), limiting throughput despite > significant free hardware resources. > > Because the WAL writes of a single RS all go sequentially to one HDFS > file, we assumed that we could improve throughput by separating writes to > more WAL files and more HDs. To do this we ran multiple region servers on > each node. > > The scaling wasn't linear(we were in no way increasing hardware, just the > number of regionservers), but we are now getting significantly more > throughput. > I would personally not say that this is a great approach to have to take, > it would generally be better to build more smaller servers which will thus > not limit themselves by trying to put a lot of data per server through a > single WAL file. > > Of course there may be another solution to this that I'm not aware of? If > so I'd love to hear it. >
-
Re: 0.92 and Read/writes not scalingJuhani Connolly 2012-04-05, 03:02
done, thanks for pointing me to that
On 04/05/2012 11:43 AM, Ted Yu wrote: > Juhani: > Thanks for sharing your results. > > Do you mind putting the summary on HBASE-5699: Run with> 1 WAL in > HRegionServer ? > > On Wed, Apr 4, 2012 at 6:45 PM, Juhani Connolly< > [EMAIL PROTECTED]> wrote: > >> another quick update on stuff: >> >> since moving back to hdfs 0.20.2 (with hbase still at 0.92), we found that >> while we made significant gains in throughput, that most of our >> regionservers IPC threads were stuck somewhere in HWal.append(out of 50, 42 >> were in append, of which 20 were in sync), limiting throughput despite >> significant free hardware resources. >> >> Because the WAL writes of a single RS all go sequentially to one HDFS >> file, we assumed that we could improve throughput by separating writes to >> more WAL files and more HDs. To do this we ran multiple region servers on >> each node. >> >> The scaling wasn't linear(we were in no way increasing hardware, just the >> number of regionservers), but we are now getting significantly more >> throughput. >> I would personally not say that this is a great approach to have to take, >> it would generally be better to build more smaller servers which will thus >> not limit themselves by trying to put a lot of data per server through a >> single WAL file. >> >> Of course there may be another solution to this that I'm not aware of? If >> so I'd love to hear it. >>
-
Re: 0.92 and Read/writes not scalingTodd Lipcon 2012-04-14, 03:02
To close the loop on this thread, we were able to track down the
issue. See https://issues.apache.org/jira/browse/HDFS-3280 - just committed it in HDFS. It's a simple patch if you want to patch your own build. Otherwise this should show up in CDH4 nightly builds tonight, and I think in CDH4b2 as well. If you want to patch on the HBase side, you can edit HLog.java to remove the checks for the "sync" method, and have it only call "hflush". It's only the compatibility path that caused the problem. Thanks -Todd On Wed, Apr 4, 2012 at 8:02 PM, Juhani Connolly <[EMAIL PROTECTED]> wrote: > done, thanks for pointing me to that > > > On 04/05/2012 11:43 AM, Ted Yu wrote: >> >> Juhani: >> Thanks for sharing your results. >> >> Do you mind putting the summary on HBASE-5699: Run with> 1 WAL in >> HRegionServer ? >> >> On Wed, Apr 4, 2012 at 6:45 PM, Juhani Connolly< >> [EMAIL PROTECTED]> wrote: >> >>> another quick update on stuff: >>> >>> since moving back to hdfs 0.20.2 (with hbase still at 0.92), we found >>> that >>> while we made significant gains in throughput, that most of our >>> regionservers IPC threads were stuck somewhere in HWal.append(out of 50, >>> 42 >>> were in append, of which 20 were in sync), limiting throughput despite >>> significant free hardware resources. >>> >>> Because the WAL writes of a single RS all go sequentially to one HDFS >>> file, we assumed that we could improve throughput by separating writes to >>> more WAL files and more HDs. To do this we ran multiple region servers on >>> each node. >>> >>> The scaling wasn't linear(we were in no way increasing hardware, just >>> the >>> number of regionservers), but we are now getting significantly more >>> throughput. >>> I would personally not say that this is a great approach to have to take, >>> it would generally be better to build more smaller servers which will >>> thus >>> not limit themselves by trying to put a lot of data per server through a >>> single WAL file. >>> >>> Of course there may be another solution to this that I'm not aware of? If >>> so I'd love to hear it. >>> > -- Todd Lipcon Software Engineer, Cloudera
-
Re: 0.92 and Read/writes not scalingStack 2012-04-14, 04:06
On Fri, Apr 13, 2012 at 8:02 PM, Todd Lipcon <[EMAIL PROTECTED]> wrote:
> If you want to patch on the HBase side, you can edit HLog.java to > remove the checks for the "sync" method, and have it only call > "hflush". It's only the compatibility path that caused the problem. > You mean change the order here boss? @Override public void sync() throws IOException { if (this.syncFs != null) { try { this.syncFs.invoke(this.writer, HLog.NO_ARGS); } catch (Exception e) { throw new IOException("Reflection", e); } } else if (this.hflush != null) { try { this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS); } catch (Exception e) { throw new IOException("Reflection", e); } } } Call hflush if its available ahead of syncFs? Seems like we should get this in all around. I can do it. Good stuff, St.Ack
-
Re: 0.92 and Read/writes not scalingJonathan Hsieh 2012-04-14, 12:43
Stack
That approach should fix the problem as well. Some updates from my previous comments -- * The problems I encountered initially against cdh3u3 was a misconfiguration on my part (I had a stress config on for that test, when removed cdh3u3 performed well). * Nagle modifications didn't make a significant difference. * Matteo wrote a new micro benchmark that just focused on the HLog, we'll should be able to get this upstream. Jon. On Fri, Apr 13, 2012 at 9:06 PM, Stack <[EMAIL PROTECTED]> wrote: > On Fri, Apr 13, 2012 at 8:02 PM, Todd Lipcon <[EMAIL PROTECTED]> wrote: > > If you want to patch on the HBase side, you can edit HLog.java to > > remove the checks for the "sync" method, and have it only call > > "hflush". It's only the compatibility path that caused the problem. > > > > You mean change the order here boss? > > > @Override > public void sync() throws IOException { > if (this.syncFs != null) { > try { > this.syncFs.invoke(this.writer, HLog.NO_ARGS); > } catch (Exception e) { > throw new IOException("Reflection", e); > } > } else if (this.hflush != null) { > try { > this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS); > } catch (Exception e) { > throw new IOException("Reflection", e); > } > } > } > > > Call hflush if its available ahead of syncFs? > > Seems like we should get this in all around. I can do it. > > Good stuff, > St.Ack > -- // Jonathan Hsieh (shay) // Software Engineer, Cloudera // [EMAIL PROTECTED]
-
Re: 0.92 and Read/writes not scalingTodd Lipcon 2012-04-15, 06:28
On Fri, Apr 13, 2012 at 9:06 PM, Stack <[EMAIL PROTECTED]> wrote:
> On Fri, Apr 13, 2012 at 8:02 PM, Todd Lipcon <[EMAIL PROTECTED]> wrote: >> If you want to patch on the HBase side, you can edit HLog.java to >> remove the checks for the "sync" method, and have it only call >> "hflush". It's only the compatibility path that caused the problem. >> > > You mean change the order here boss? Yep - invoking hflush instead of syncfs should fix the issue on older 0.23.x/CDH4 builds, I think (though I didn't test it). Going forward it won't matter though. FYI I verified that the fix made it into our nightly CDH4 build last night (0.23.1+360) -Todd > > > @Override > public void sync() throws IOException { > if (this.syncFs != null) { > try { > this.syncFs.invoke(this.writer, HLog.NO_ARGS); > } catch (Exception e) { > throw new IOException("Reflection", e); > } > } else if (this.hflush != null) { > try { > this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS); > } catch (Exception e) { > throw new IOException("Reflection", e); > } > } > } > > > Call hflush if its available ahead of syncFs? > > Seems like we should get this in all around. I can do it. > > Good stuff, > St.Ack -- Todd Lipcon Software Engineer, Cloudera |