[ https://issues.apache.org/jira/browse/HBASE-12948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14317341#comment-14317341 ]
hongyu bi commented on HBASE-12948: ----------------------------------- Thanks [~stack] for review:) as for open another issue to match the get's way ,i have a concern that rolling upgrade support may be the tricky part.I just thought two options: option#1: create a treemap to match the get's behaviour from client side but sending a List finally which may need a copy? option#2: prohibit multi mutations on the same column in one mutation call. both options seems not that elegant ,just a quick thought any suggestion are welcome thanks > Increment#addColumn on the same column multi times produce wrong result > ------------------------------------------------------------------------ > > Key: HBASE-12948 > URL: https://issues.apache.org/jira/browse/HBASE-12948 > Project: HBase > Issue Type: Bug > Components: Client, regionserver > Reporter: hongyu bi > Assignee: hongyu bi > Priority: Critical > Attachments: 12948-v2.patch, HBASE-12948-0.99.2-v1.patch, > HBASE-12948-v0.patch, HBASE-12948.patch > > > Case: > Initially get('row1'): > rowkey=row1 value=1 > run: > Increment increment = new Increment(Bytes.toBytes("row1")); > for (int i = 0; i < N; i++) { > increment.addColumn(Bytes.toBytes("cf"), Bytes.toBytes("c"), 1) > } > hobi.increment(increment); > get('row1'): > if N=1 then result is 2 else if N>1 the result will always be 1 > Cause: > https://issues.apache.org/jira/browse/HBASE-7114 let increment extent > mutation which change familyMap from NavigableMap to List, so from client > side, we can buffer many edits on the same column; > However, HRegion#increment use idx to iterate the get's results, here > results.size<family.value().size if N>1,so the latter edits on the same > column won't match the condition {idx < results.size() && > CellUtil.matchingQualifier(results.get(idx), kv) }, meantime the edits share > the same mvccVersion ,so this case happen. > Fix: > according to the put/delete#add on the same column behaviour , > fix from server side: process "last edit wins on the same column" inside > HRegion#increment to maintenance HBASE-7114's extension and keep the same > result from 0.94. -- This message was sent by Atlassian JIRA (v6.3.4#6332)