Re: Re: Poor performance of table Increment
I had the same issue. I ended up writing a coprocessor that did this under the BatchMutate hood using a special attribute in the Put operation, and that increased performance significantly to be almost equal to the regular batchMutate(Put) version. On Sat, Jul 9, 2016 at 7:59 AM, 陆巍 wrote: > The performance of batch(Increment) was poor, which is why I do the test. > > > I use spark stream to process data per one second (for about 50 records) > and use HBase for aggregations (sum, max, min etc). All the spark executors > are running on the same machines as the RegionServers, so the network env > is fine. > In the hbase caller, there are Put, CheckAndPut, and Increment operations. > Put and Increment are called thru Table.batch(...). > Put and CheckAndPut cost less was around 50ms, but Increment costs around > 1 minute. > > > > At 2016-07-09 22:34:05, la...@apache.org wrote: > >Note that you're doing single RPCs for each Increment. You should batch > them (see HTable.batch(...)) > > > >What version of HBase? What's the ping time between the client and server > machines?Is Nagle's disables (tcpnodelay set to true)? Otherwise each RPC > will take at least 45 ms. > > > >Lastly you'll likely hit the same server for every request in a > microbenchmark like this. > > > >You are also not actually incrementing anything since each one is a new > row (I realize this is just a test),so to test whether this is related to > Increment, can you replace the Increment by Put and see how long that takes? > > > >-- Lars > > > > From: 陆巍 > > To: user@hbase.apache.org > > Sent: Saturday, July 9, 2016 3:14 AM > > Subject: Poor performance of table Increment > > > >Hi, > > > >I had a test for Increment operation, and find the performance is really > bad: 94809ms for 1000 increment operaions. > >The testing cluster is pretty small with only three region servers. > > > >Below is the code for testing. Is there any bentchmark result for htable > increment? Thanks! > > > >import java.io.IOException; > >import java.util.Random; > > > >import org.apache.hadoop.conf.Configuration; > >import org.apache.hadoop.hbase.HBaseConfiguration; > >import org.apache.hadoop.hbase.TableName; > >import org.apache.hadoop.hbase.client.Connection; > >import org.apache.hadoop.hbase.client.ConnectionFactory; > >import org.apache.hadoop.hbase.client.Increment; > >import org.apache.hadoop.hbase.client.Table; > >import org.apache.hadoop.hbase.util.Bytes; > > > >class HBaseTableFactory { > >private Configuration config = HBaseConfiguration.create(); > >private Connection connection = null; > > > >public Table createTable(String tableName) throws IOException { > >if (connection == null) { > >connection = ConnectionFactory.createConnection(config); > >} > >return create(tableName, config); > >} > > > >protected Table create(String tableName, Configuration config) > >throws IOException { > >return connection.getTable(TableName.valueOf(tableName)); > >} > > > >} > > > >public class TestBytes { > >public static void main(String[] args) throws IOException { > >long t1 = System.currentTimeMillis(); > >Table table = new HBaseTableFactory().createTable("xyz"); > >for (int i = 0; i < 1000; i++) { > > > >byte[] row = Bytes.toBytes("row-" + i); > >Increment p = new Increment(row); > >byte[] f = Bytes.toBytes("cf1"); > >byte[] q = Bytes.toBytes("q1"); > >long v = 1L; > >p.addColumn(f, q, v); > >table.increment(p); > >System.out.println(i); > >} > >table.close(); > >System.out.println(System.currentTimeMillis() - t1); > >} > >} > > > > > > >
Re: Poor performance of table Increment
Note that you're doing single RPCs for each Increment. You should batch them (see HTable.batch(...)) What version of HBase? What's the ping time between the client and server machines?Is Nagle's disables (tcpnodelay set to true)? Otherwise each RPC will take at least 45 ms. Lastly you'll likely hit the same server for every request in a microbenchmark like this. You are also not actually incrementing anything since each one is a new row (I realize this is just a test),so to test whether this is related to Increment, can you replace the Increment by Put and see how long that takes? -- Lars From: 陆巍 To: user@hbase.apache.org Sent: Saturday, July 9, 2016 3:14 AM Subject: Poor performance of table Increment Hi, I had a test for Increment operation, and find the performance is really bad: 94809ms for 1000 increment operaions. The testing cluster is pretty small with only three region servers. Below is the code for testing. Is there any bentchmark result for htable increment? Thanks! import java.io.IOException; import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; class HBaseTableFactory { private Configuration config = HBaseConfiguration.create(); private Connection connection = null; public Table createTable(String tableName) throws IOException { if (connection == null) { connection = ConnectionFactory.createConnection(config); } return create(tableName, config); } protected Table create(String tableName, Configuration config) throws IOException { return connection.getTable(TableName.valueOf(tableName)); } } public class TestBytes { public static void main(String[] args) throws IOException { long t1 = System.currentTimeMillis(); Table table = new HBaseTableFactory().createTable("xyz"); for (int i = 0; i < 1000; i++) { byte[] row = Bytes.toBytes("row-" + i); Increment p = new Increment(row); byte[] f = Bytes.toBytes("cf1"); byte[] q = Bytes.toBytes("q1"); long v = 1L; p.addColumn(f, q, v); table.increment(p); System.out.println(i); } table.close(); System.out.println(System.currentTimeMillis() - t1); } }
Re: Poor performance of table Increment
Which release of hbase are you using ? Does it include HBASE-15213 ? Thanks On Sat, Jul 9, 2016 at 3:14 AM, 陆巍 wrote: > Hi, > > I had a test for Increment operation, and find the performance is really > bad: 94809ms for 1000 increment operaions. > The testing cluster is pretty small with only three region servers. > > Below is the code for testing. Is there any bentchmark result for htable > increment? Thanks! > > import java.io.IOException; > import java.util.Random; > > import org.apache.hadoop.conf.Configuration; > import org.apache.hadoop.hbase.HBaseConfiguration; > import org.apache.hadoop.hbase.TableName; > import org.apache.hadoop.hbase.client.Connection; > import org.apache.hadoop.hbase.client.ConnectionFactory; > import org.apache.hadoop.hbase.client.Increment; > import org.apache.hadoop.hbase.client.Table; > import org.apache.hadoop.hbase.util.Bytes; > > class HBaseTableFactory { > private Configuration config = HBaseConfiguration.create(); > private Connection connection = null; > > public Table createTable(String tableName) throws IOException { > if (connection == null) { > connection = ConnectionFactory.createConnection(config); > } > return create(tableName, config); > } > > protected Table create(String tableName, Configuration config) > throws IOException { > return connection.getTable(TableName.valueOf(tableName)); > } > > } > > public class TestBytes { > public static void main(String[] args) throws IOException { > long t1 = System.currentTimeMillis(); > Table table = new HBaseTableFactory().createTable("xyz"); > for (int i = 0; i < 1000; i++) { > > byte[] row = Bytes.toBytes("row-" + i); > Increment p = new Increment(row); > byte[] f = Bytes.toBytes("cf1"); > byte[] q = Bytes.toBytes("q1"); > long v = 1L; > p.addColumn(f, q, v); > table.increment(p); > System.out.println(i); > } > table.close(); > System.out.println(System.currentTimeMillis() - t1); > } > } > >