hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "jiraposter@reviews.apache.org (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-2214) Do HBASE-1996 -- setting size to return in scan rather than count of rows -- properly
Date Tue, 01 May 2012 07:57:55 GMT

    [ https://issues.apache.org/jira/browse/HBASE-2214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13265702#comment-13265702
] 

jiraposter@reviews.apache.org commented on HBASE-2214:
------------------------------------------------------



bq.  On 2012-04-30 20:03:34, Michael Stack wrote:
bq.  > Where are we checking the size of the result made so far?  I don't see it in the
below.  I'd expect it inside in the RegionScanner.  Any chance of a test?  Otherwise, patch
looks great.
bq.  
bq.  ferdy wrote:
bq.      Please see the testing method in the Testing Done field above. (Not sure where to
add a test in the project).
bq.      
bq.      Thanks for the feedback.

(One more thing, it works because of the previous work done in HBASE-1996. But it's arguably
not the best way to implement it.)


- ferdy


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/4726/#review7383
-----------------------------------------------------------


On 2012-05-01 07:50:07, ferdy wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/4726/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2012-05-01 07:50:07)
bq.  
bq.  
bq.  Review request for hbase and Ted Yu.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-2214 per scan max buffersize.
bq.  
bq.  
bq.  This addresses bug HBASE-2214.
bq.      https://issues.apache.org/jira/browse/HBASE-2214
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    /src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java 1330680 
bq.    /src/main/java/org/apache/hadoop/hbase/client/Scan.java 1330680 
bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java 1330680 
bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java 1330680 
bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java 1330680

bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java 1330680

bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java 1330680

bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java 1330680 
bq.    /src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java 1330680

bq.    /src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java 1332544 
bq.    /src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1332544 
bq.    /src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java 1332544 
bq.    /src/main/java/org/apache/hadoop/hbase/regionserver/RegionServer.java 1332544 
bq.    /src/main/protobuf/Client.proto 1330680 
bq.    /src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java 1332544

bq.  
bq.  Diff: https://reviews.apache.org/r/4726/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  It works when running this test:
bq.  
bq.  
bq.      new HBaseTestingUtility(conf).startMiniCluster();
bq.   
bq.      HBaseAdmin admin = new HBaseAdmin(conf);
bq.      if (!admin.tableExists("test")) {
bq.        HTableDescriptor tableDesc = new HTableDescriptor("test");
bq.        tableDesc.addFamily(new HColumnDescriptor("fam"));
bq.        admin.createTable(tableDesc);
bq.      }
bq.      
bq.      
bq.      HTable table = new HTable(conf, "test");
bq.      Put put; 
bq.      
bq.      put = new Put(Bytes.toBytes("row1"));
bq.      put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual1"),Bytes.toBytes("val1"));
bq.      table.put(put);
bq.      
bq.      put = new Put(Bytes.toBytes("row2"));
bq.      put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual2"),Bytes.toBytes("val2"));
bq.      table.put(put);
bq.      
bq.      put = new Put(Bytes.toBytes("row3"));
bq.      put.add(Bytes.toBytes("fam"),Bytes.toBytes("qual3"),Bytes.toBytes("val3"));
bq.      table.put(put);
bq.      
bq.      table.flushCommits();
bq.      //put a logging statement to ClientScanner#next() to see the effect.
bq.      {
bq.        System.out.println("returns all rows at once because of the caching");
bq.        Scan scan = new Scan();
bq.        scan.setCaching(100);
bq.        ResultScanner scanner = table.getScanner(scan);
bq.        scanner.next(100);
bq.      }
bq.      {
bq.        System.out.println("returns one row at a time because of the maxResultSize");
bq.        Scan scan = new Scan();
bq.        scan.setCaching(100);
bq.        scan.setMaxResultSize(1);
bq.        ResultScanner scanner = table.getScanner(scan);
bq.        scanner.next(100);
bq.      }
bq.  
bq.  
bq.  See output:
bq.  
bq.  returns all rows at once because of the caching
bq.  2012-04-25 22:18:47,494 DEBUG [main] client.ClientScanner(94): Creating scanner over
test starting at key ''
bq.  2012-04-25 22:18:47,494 DEBUG [main] client.ClientScanner(206): Advancing internal scanner
to startKey at ''
bq.  2012-04-25 22:18:47,499 DEBUG [main] client.ClientScanner(323): Rows returned 3
bq.  2012-04-25 22:18:47,502 DEBUG [main] client.ClientScanner(193): Finished with scanning
at {NAME => 'test,,1335385126388.ed23a82f3d6ca2eab571918843796259.', STARTKEY => '',
ENDKEY => '', ENCODED => ed23a82f3d6ca2eab571918843796259,}
bq.  returns one row at a time because of the maxResultSize
bq.  2012-04-25 22:18:47,504 DEBUG [main] client.ClientScanner(94): Creating scanner over
test starting at key ''
bq.  2012-04-25 22:18:47,505 DEBUG [main] client.ClientScanner(206): Advancing internal scanner
to startKey at ''
bq.  2012-04-25 22:18:47,514 DEBUG [main] client.ClientScanner(323): Rows returned 1
bq.  2012-04-25 22:18:47,517 DEBUG [main] client.ClientScanner(323): Rows returned 1
bq.  2012-04-25 22:18:47,522 DEBUG [main] client.ClientScanner(323): Rows returned 1
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ferdy
bq.  
bq.


                
> 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-v6.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

        

Mime
View raw message