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

stack commented on HBASE-8763:
------------------------------

Looking at 5.1:

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.

Why would the below take a list of kvs to appendNoSyncNoAppend?  When would it 
ever make sense passing a list of kvs?  (Hmm... I think I see why -- when we 
want to just set an mvcc on the KV though we are not appending it -- is that 
right?  If so, a comment on the @param would help)

In the appendNoSyncNoAppend we make HLogKey.  We call System.currentTimeMillis. 
 This edit is never appended.  Can we NOT call System.currentTimeMillis?  Just 
pass a -1 or something instead?  Or can we make a noop HLogKey defined as a 
static in HLogKey and just use it every time rather than create a new one each 
time through?  Just as we have WALEdit.EMPTY_WALEDIT?

On beginMemstoreInsert, why take a value at all?  Wny not just return the 
WriteEntry that has special value for the write number?  If we ever try to use 
this number advancing the read point, throw exceptions?  Remove the current 
beginMemstoreInsert that does not take an entry?  I see that in your new 
method, waitForPreviousTransactionsComplete, you put something into the mvcc 
queue w/ a HLog.NO_SEQUENCE_ID and wait for this edit to go around so you can 
be sure queue is cleared.  So you have second use for special mvcc/sequenceid 
number.  Should the NO_SEQUENCE_ID be it?  and you just use it when 
beginMemstoreInsert is called setting it into the WriteEntry?  Should the 
number even come from HLog?  Could it be private to this class?
  
When we do waitForPreviousTransactionsComplete, does WriteEntry have a 
writeNumber set? Or is it NO_SEQUENCE_ID?  If it is the latter, yeah, just 
change beginMemstoreInsert to not take a param, or at least, not take this 
particular one because it is means of asking for a special behavior.  If the 
writeNumber is set, where does that happen?

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!)

I am not clear still on why the below is ok up in HRegion#doMiniBatchMutation 
(Do we need the MutableLong here still? Why not just set the sequenceid into 
beginMemstoreInsertWithSeqNum and you are doing this when you use it   
kv.setMvccVersion(mvccNum.longValue());)

      mvccNum.setValue(this.sequenceId.incrementAndGet());

At a minimum it needs a comment explaining why (Sorry if I am being dense here).

You know why we add to memstore first before WAL?  For speed IIRC.  I should go 
research it.  This rollback stuff could be tricky.

So then here:

        mvccNum.setValue(this.sequenceId.incrementAndGet());
        w = mvcc.beginMemstoreInsertWithSeqNum(mvccNum);

We are getting a seqid and setting it as write number. We have not yet gone on 
the ring buffer.  Every edit is getting a write number like this?  MVCC read 
number happens only after the WAL append has happened.

Man, the mvcc stuff should be redone w/ disruptor.  Looks like ideal disruptor 
case.

StoreFlusher change no longer needed?

Can these be lists of Cells rather than   private final transient 
List<KeyValue> memstoreKVs;?  You can do cell.setMvccVersion.

Why this?

  long appendNoSync(HTableDescriptor htd, HRegionInfo info, HLogKey key, 
WALEdit edits,
      AtomicLong sequenceId, boolean inMemstore, List<KeyValue> memstoreKVs)

Unnecssary import in HLogSplitter?  Unnecessary change in WALEdit?





Are we not passing the KVs twice?  Once in WALEdits and then again in this new 
memstoreKVs argument?

I'm running tests now to see what this patch does for performance.  After our 
chat yesterday, yes, I see, it should not have much of an impact (especially 
looking at what you did in FSHLog).  That'd be cool.

I'm excited about this patch coming in.  Great work Mr. Zhong.





















> [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)

Reply via email to