You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org> on 2012/05/01 09:51:47 UTC
[jira] [Commented] (HBASE-2214) Do HBASE-1996 -- setting size to
return in scan rather than count of rows -- properly
[ https://issues.apache.org/jira/browse/HBASE-2214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13265699#comment-13265699 ]
jiraposter@reviews.apache.org commented on HBASE-2214:
------------------------------------------------------
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/4726/
-----------------------------------------------------------
(Updated 2012-05-01 07:50:07.133923)
Review request for hbase and Ted Yu.
Changes
-------
v5 (updated to head and removed debug "rows returned" statement)
Summary
-------
HBASE-2214 per scan max buffersize.
This addresses bug HBASE-2214.
https://issues.apache.org/jira/browse/HBASE-2214
Diffs (updated)
-----
/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java 1330680
/src/main/java/org/apache/hadoop/hbase/client/Scan.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java 1330680
/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java 1330680
/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 1332544
/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1332544
/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java 1332544
/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServer.java 1332544
/src/main/protobuf/Client.proto 1330680
/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java 1332544
Diff: https://reviews.apache.org/r/4726/diff
Testing (updated)
-------
It works when running this test:
new HBaseTestingUtility(conf).startMiniCluster();
HBaseAdmin admin = new HBaseAdmin(conf);
if (!admin.tableExists("test")) {
HTableDescriptor tableDesc = new HTableDescriptor("test");
tableDesc.addFamily(new HColumnDescriptor("fam"));
admin.createTable(tableDesc);
}
HTable table = new HTable(conf, "test");
Put put;
put = new Put(Bytes.toBytes("row1"));
put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual1"),Bytes.toBytes("val1"));
table.put(put);
put = new Put(Bytes.toBytes("row2"));
put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual2"),Bytes.toBytes("val2"));
table.put(put);
put = new Put(Bytes.toBytes("row3"));
put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual3"),Bytes.toBytes("val3"));
table.put(put);
table.flushCommits();
//put a logging statement to ClientScanner#next() to see the effect.
{
System.out.println("returns all rows at once because of the caching");
Scan scan = new Scan();
scan.setCaching(100);
ResultScanner scanner = table.getScanner(scan);
scanner.next(100);
}
{
System.out.println("returns one row at a time because of the maxResultSize");
Scan scan = new Scan();
scan.setCaching(100);
scan.setMaxResultSize(1);
ResultScanner scanner = table.getScanner(scan);
scanner.next(100);
}
See output:
returns all rows at once because of the caching
2012-04-25 22:18:47,494 DEBUG [main] client.ClientScanner(94): Creating scanner over test starting at key ''
2012-04-25 22:18:47,494 DEBUG [main] client.ClientScanner(206): Advancing internal scanner to startKey at ''
2012-04-25 22:18:47,499 DEBUG [main] client.ClientScanner(323): Rows returned 3
2012-04-25 22:18:47,502 DEBUG [main] client.ClientScanner(193): Finished with scanning at {NAME => 'test,,1335385126388.ed23a82f3d6ca2eab571918843796259.', STARTKEY => '', ENDKEY => '', ENCODED => ed23a82f3d6ca2eab571918843796259,}
returns one row at a time because of the maxResultSize
2012-04-25 22:18:47,504 DEBUG [main] client.ClientScanner(94): Creating scanner over test starting at key ''
2012-04-25 22:18:47,505 DEBUG [main] client.ClientScanner(206): Advancing internal scanner to startKey at ''
2012-04-25 22:18:47,514 DEBUG [main] client.ClientScanner(323): Rows returned 1
2012-04-25 22:18:47,517 DEBUG [main] client.ClientScanner(323): Rows returned 1
2012-04-25 22:18:47,522 DEBUG [main] client.ClientScanner(323): Rows returned 1
Thanks,
ferdy
> Do HBASE-1996 -- setting size to return in scan rather than count of rows -- properly
> -------------------------------------------------------------------------------------
>
> Key: HBASE-2214
> URL: https://issues.apache.org/jira/browse/HBASE-2214
> Project: HBase
> Issue Type: New Feature
> Reporter: stack
> Assignee: Ferdy Galema
> Attachments: HBASE-2214-0.94.txt, HBASE-2214-v4.txt, HBASE-2214-v5.txt, HBASE-2214_with_broken_TestShell.txt
>
>
> The notion that you set size rather than row count specifying how many rows a scanner should return in each cycle was raised over in hbase-1966. Its a good one making hbase "regular" though the data under it may vary. HBase-1966 was committed but the patch was constrained by the fact that it needed to not change RPC interface. This issue is about doing hbase-1966 for 0.21 in a clean, unconstrained way.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira