[ https://issues.apache.org/jira/browse/HBASE-8763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14019021#comment-14019021 ]
Jeffrey Zhong commented on HBASE-8763: -------------------------------------- [~saint....@gmail.com] Thanks for the review and good comments. {quote} Why return a Pair? We are passing in the Cell? Can't caller use passed in Cell and just leave addTo returning long as before (I am missing something I know... pardon my being slow). We are making a new object each time we add to memstore, the Pair. {quote} That's the issue on how we add cells to memestore today. Basically it does copy the passed in Cells and put newly created Cells into memstore. So I have to return those newly added Cells to update mvcc later(because JAVA doesn't support reference to a primitive type). In future, we shouldn't make copies when add cells into memstore. {quote} If so, a comment on the @param would help {quote} Sure. I'll do that. {quote} create a new one each time through? Just as we have WALEdit.EMPTY_WALEDIT? {quote} Good point! I'll do that. {quote} On beginMemstoreInsert, why take a value at all {quote} It's used by beginMemstoreInsertWithSeqNum. One way is that I can let beginMemstoreInsert to call beginMemstoreInsertWithSeqNum instead. After this, I think it can address several following comments. {quote} NO_SEQUENCE_ID Should be a define in your new SequenceId interface? A comment on wny you do 'w = null;' would be helpful in flush: e.g. "Set to null to indicate success" Change name of memstoreKVs to be memstoreCells (be forward thinking!) {quote} Ok, I'll change that. {quote} Do we need the MutableLong here still? {quote} The reason is that I want to hide the fact of bumping 1 billion number inside this function and keep the bumping in one place. I could define a constant for this purpose {quote} Can these be lists of Cells rather than private final transient List<KeyValue> memstoreKVs;? You can do cell.setMvccVersion. {quote} We don't have setMvccVersion in Cell interface. Do you want to create one? {quote} Are we not passing the KVs twice? Once in WALEdits and then again in this new memstoreKVs argument? {quote} My firstly version(not published) is using KVs in WALEdits while HRegion#doProcessRowWithTimeout let clients to create WALEdits so it's impossible to merge those two lists. {quote} Man, the mvcc stuff should be redone w/ disruptor. Looks like ideal disruptor case. {quote} The issue lands in we have different durability modes so the disruptor way maybe hard. As in the earlier review thread, we could remove mvcc writeQueue while it needs to keep the sequence id of the last unflushed edit with sync_wal durability. Let the optimization be done later. > [BRAINSTORM] Combine MVCC and SeqId > ----------------------------------- > > Key: HBASE-8763 > URL: https://issues.apache.org/jira/browse/HBASE-8763 > Project: HBase > Issue Type: Improvement > Components: regionserver > Reporter: Enis Soztutar > Assignee: Jeffrey Zhong > Priority: Critical > Attachments: HBase MVCC & LogSeqId Combined.pdf, > hbase-8736-poc.patch, hbase-8763-poc-v1.patch, hbase-8763-v1.patch, > hbase-8763-v2.patch, hbase-8763-v3.patch, hbase-8763-v4.patch, > hbase-8763-v5.1.patch, hbase-8763-v5.patch, hbase-8763_wip1.patch > > > HBASE-8701 and a lot of recent issues include good discussions about mvcc + > seqId semantics. It seems that having mvcc and the seqId complicates the > comparator semantics a lot in regards to flush + WAL replay + compactions + > delete markers and out of order puts. > Thinking more about it I don't think we need a MVCC write number which is > different than the seqId. We can keep the MVCC semantics, read point and > smallest read points intact, but combine mvcc write number and seqId. This > will allow cleaner semantics + implementation + smaller data files. > We can do some brainstorming for 0.98. We still have to verify that this > would be semantically correct, it should be so by my current understanding. -- This message was sent by Atlassian JIRA (v6.2#6252)