[
https://issues.apache.org/jira/browse/HBASE-8763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14000455#comment-14000455
]
Jeffrey Zhong commented on HBASE-8763:
--------------------------------------
Thanks [[email protected]] for the good comments! I'll make corresponding
changes in my next patch. Please see my answers below:
{quote}
Hmm... maybe we should make the writeQueue a disruptor too? Later.
{quote}
That's a good idea. I could also remove the writeQueue even without disruptor.
Basically return the previous highestUnsyncedSequence from disruptor consumer
code where we stampRegionSequenceId in FSHLog#append() before current wal
append, inside syncOrDefer we will wait for this previous
highestUnsyncedSequence for skip_wal & async_wal case. Let's do it later in a
separate JIRA.
{quote}
Should beginMemstoreInsert assign a write mvcc number at all?
{quote}
No need to assign a write number and I could pass 0 here. The main purpose for
the call is inserting a marker so that later code can wait all mvcc
transactions before marker complete. this.sequenceId.incrementAndGet() is just
a way to pass some "good" number without messing up the memstore read point
later by mvcc.advanceMemstore(w) call.
I'll pass HLog.NO_SEQUENCE_ID value then.
{quote}
Misspelling: waitForPreviousTransactoinsComplete
{quote}
Good catch.
{quote}
Call this appendNoSyncFakedWALEdit appendNoSyncNoAppend
{quote}
Ok. I'll make the change
{quote}
It is convenient updating once and all related get the update. Is that the only
reason?
{quote}
It's not only for convenient but for correctness. Let's say we have two updates
coming in the order of c1, c2 while c1 could have seqId=2 and c2 could have
seqId=1. As you know wal syncer syncs all available pending appends and it's
likely both wal entries for e1 and e2 are synced at same time. Therefore, the
mvcc for c1 will advance memstore read point to 2. Since MutableLong is used
here, so we know for sure c1's MVCC has been updated to 1 before c2 gets its
seqid=2 otherwise we could end up with the situation that memstore read point
has been set to 2 while c1's mvcc in menstore hasn't been updated yet depending
on the caller thread scheduling.(Unless I pass MVCC into ring buffer where we
keep the reference of all new KVs and update mvcc for them in disruptor
consumer code) .
{quote}
Set top bits rather than add a big number?: + curSeqNum.setValue(originalVal +
1000000000);
{quote}
How about the case if the first bit is already used for a large sequence
number? In theory, I only need to bump the number to 2 * the number of rpc
handlers because the writeQueue will be blocked if current mvcc isn't complete.
The big number here is just to be safe.
{quote}
Do we need to have MemStore know about HLogKeys?
Could we do the wait in the WAL system before we call
completeMemstoreInsertWithSeqNum passing in the sequence id to use? Could the
consumer on the ring buffer call the Memstore. completeMemstoreInsertWithSeqNum?
{quote}
Memstore doesn't know hlogkey but MVCC. I can do it outside of MVCC but that
requires all places calling completeMemstoreInsertWithSeqNum need to remember
to call the waitForLogSequence. It may leave hole in the future.
> [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_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)