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

ramkrishna.s.vasudevan commented on HBASE-10531:
------------------------------------------------

bq.But I'd say KV methods should take KVs, not Cells? If it takes Cells, could 
be comparing a non-KV and it could actually work? Is this what we want? Maybe 
this is just uglyness left over from how KV has been used/abused up to this? 
But I'm thinking these compares would be Cell compares out in a CellUtil or 
CellCompare class?
See HBASE-10532.  All the above compares have been moved over there.  But for 
this JIRA I have still maintained things as KVComparator. Did not want to 
change the KVComparator part here. I could change that also and call CellUtil 
or CellComparator. Let me see how to handle that here.

bq.Shouldn't this be unsupportedoperationexception in your new key only class?
I think yes.  But I faced some issue, hence added it. Let me check it once 
again in the next patch.

bq.Why we have to create new key when we pass to a comparator?
I will add suitable comments.
bq.Should it be an offset? We do this '0' in a few places.
Where ever offset is needed I have used that. whereever 0 is needed I have used 
0.  I can cross check once again.
bq.So, this is the replacement: seekToKeyInBlock ? Purge the old stuff!!!!
I did not do that just for sake of easy review. Will purge all the duplicate 
code.
bq.{ The array of byte arrays has Cells in it or it seems KVs? Will it always 
be arrays of byte arrays?
I would suggest in the follow up JIRAs we can change to Cells? rather than 
byte[]

All the last comments are about the refactoring part. I have not removed the 
old code and hence you say them. I can remove them too.  testReseek() i will 
change to work with Cells, but thing to be noted is that previously it was 
working with RawBytecomparator, am planning to change to KVComparator only.  
Same with TestSeekTo.

> Revisit how the key byte[] is passed to HFileScanner.seekTo and reseekTo
> ------------------------------------------------------------------------
>
>                 Key: HBASE-10531
>                 URL: https://issues.apache.org/jira/browse/HBASE-10531
>             Project: HBase
>          Issue Type: Sub-task
>            Reporter: ramkrishna.s.vasudevan
>            Assignee: ramkrishna.s.vasudevan
>             Fix For: 0.99.0
>
>         Attachments: HBASE-10531.patch, HBASE-10531_1.patch, 
> HBASE-10531_2.patch
>
>
> Currently the byte[] key passed to HFileScanner.seekTo and 
> HFileScanner.reseekTo, is a combination of row, cf, qual, type and ts.  And 
> the caller forms this by using kv.getBuffer, which is actually deprecated.  
> So see how this can be achieved considering kv.getBuffer is removed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to