Skip to content

Conversation

@amaliujia
Copy link
Contributor

What changes were proposed in this pull request?

Design doc: https://docs.google.com/document/d/1uy4_ER2V6nNQJ7_5455Wz8NmI142JHPnif6Y1OdPi8E/edit?usp=sharing

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-3208

How was this patch tested?

UT

@amaliujia
Copy link
Contributor Author

I found one of the failed test is TestMiniOzoneCluster.testDNstartAfterSCM, which failed on loading RocksDB when restarting SCM. Will try investigate the root cause.

@amaliujia
Copy link
Contributor Author

cc @nandakumar131 @ChenSammi @GlenGeng

Copy link
Contributor

@GlenGeng-awx GlenGeng-awx left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. only some inline comments.

Could we rename the title to implement Ratis takeSnapshot on SCM?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we remove the 'else {}' branch ? When we will migrate to batch operations, there will be no other way to do db writes.

Copy link
Contributor Author

@amaliujia amaliujia Dec 23, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is left for MockHAManager code path in which the buffer is NULL :)

If we choose to update MockHAManager to have a sort of MockBuffer implementation, then we can move this else branch because at that time I think all existing code will access this buffer.

I can create a JIRA to track that add buffer to MockHAManager and remove this else :-)

Copy link
Contributor Author

@amaliujia amaliujia Dec 31, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

https://issues.apache.org/jira/browse/HDDS-4634

Create JIRA to add buffer in MockHAManager thus we can remove else

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Will it be better to put SCMTransactionInfo near OMTransactionInfo, under interface-storage ?

  • Can we reuse the OMTransactionInfo? For example, rename OMTransactionInfo to TransactionInfo, and use them in both OM and SCM ?

Copy link
Contributor Author

@amaliujia amaliujia Dec 23, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To reuse OMTransactionInfo, my current answer is probably no.

Though both OMTransactionInfo and SCMTransactionInfo seems are doing the same thing for now, decoupling is good when there is a need that either OMTransactionInfo or SCMTransactionInfo needs to add(drop) new(existing) fields while the other does not need. Especially at this moment that SCM snapshot is still under development.

I think we can consider this after SCM snapshot is stable. I can create a JIRA though to track the merge of two transaction info.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I'm not clear why we can not reuse OMTransactionInfo. They has the same fields.

Copy link
Contributor Author

@amaliujia amaliujia Jan 8, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we are better to reuse OMTransactionInfo after merge HDDS-2823 back to master.

Not reusing OMTransactionInfo is to try to reduce the risk that have conflicts after syncing master int HDDS-2823. Also, to reuse OMTransactionInfo, I think we'd better to update the name to TransactionInfo, which will be a source of conflicts , unless we can change it in the master branch first. And I also add some more functions in SCMTransactionInfo, if we reuse OMTransactionInfo, those new functions will also become source of conflicts.

To conclude a bit, we can reuse it after merging SCM HA back to master, the goal is try to reduce potential conflicts if there could be any.

SCM HA is in developing branch anyway and I am sure we will need to clean up lots of stuff eventually. As long as we have JIRAs to not lose track of such work.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OMTransactionInfo is written by Bharat, since SCM and OM has a quite similar requirement, we may have a talk with him, to determine whether OM and SCM can share the same TransactionInfo and the Codec. @amaliujia What do you think ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure we can do that. I will need add some changes in main branch, and then sync that to HDDS-2823 and then reuse in SCM.

I think we should do that separately as that will take time. I have created https://issues.apache.org/jira/browse/HDDS-4660 to track this effort.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we avoid using this Ctor ? We may need to change the related test cases.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same https://issues.apache.org/jira/browse/HDDS-4634.

This is because MockHAManager does not have a mock buffer implementation.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we implement the mock buffer in this PR ? Since missing this change will pollute the production code, and will also give a burden to the on-going PR of deleted block log.

@amaliujia amaliujia changed the title HDDS-3208. Implement Ratis Snapshots on SCM HDDS-3208. Implement Ratis takeSnapshot on SCM Dec 23, 2020
@amaliujia amaliujia changed the title HDDS-3208. Implement Ratis takeSnapshot on SCM HDDS-3208. SCM Snapshot Phase 1: Implement Ratis takeSnapshot on SCM Dec 23, 2020
@amaliujia
Copy link
Contributor Author

With #1733 we will be able to write tests for snapshot based on on single Ratis server SCM setup in MiniOzoneCluster.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we add some checks here to guarantee new info is greater than current lastTrxInfo?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. I will add a check or a LOG.error for that case.

Theoretically, the Ratis log will be applied with index monotonically increasing thus the new info is supposed to greater than current lastTrxInfo. But have a check or a LOG will better handle rare cases or bugs.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Contributor

@bshashikant bshashikant Jan 15, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In my opinion, we should stop the ratis server here if such a case arise. It is fatal case where we cannot move ahead from here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Such case should be considered as a bug and be fixed.

I created a JIRA to further discuss how to deal with such case: https://issues.apache.org/jira/browse/HDDS-4723

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we add a new table in SCM DB, we need a JIRA to track how to handle back compatibility during upgrade non HA SCM to HA SCM.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ChenSammi
Copy link
Contributor

@amaliujia, thanks for working on this, I left a few comments.

@amaliujia
Copy link
Contributor Author

@ChenSammi thank you! I will start to address comments since there are a bunch of actionable ones existing.

Copy link
Contributor

@linyiqun linyiqun Jan 2, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we document javadoc for this? That will be easier understood by others. At lease, we should document the purpose for this transaction buffer.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Expose currentBatchOperation has a risk, if someone commit this batch operation not by calling flush(), then applyIndex did not wrote into RocksDB.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm I think the goal here is commiting batch operations won't need to flush DB. The flushing is controlled by takeSnapshot().

Is there a case that committing will have a following flush?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm afraid if someone do not know must call flush() to sync, but sync currentBatchOperation directly, then inconsistent will happen.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you know is there a better mechanism to achieve the buffering?

I studied what OM does and that is different from SCM. OM just buffer entry and then apply entries in a batch, but SCM need to router such entry to handlers and then different handler will apply the entry.

So if we change to the way that OM is doing, there will be a good amount of refactoring needs happen, which might be not appropriate in this PR. E.g. We will need to move handlers to the buffer class and insert entries into buffer. Buffer will be the place to trouter entries to right state managers.

What do you think? I can create a PR to track such refactoring thus we won't need to expose currentBatchOperation

Copy link
Contributor

@GlenGeng-awx GlenGeng-awx Jan 8, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about return an anonymous subclass of RDBBatchOperation whose commit() will throw RuntimeException, which means you can only write to the batch, but there is no way to commit this batch ?

RDBBatchOperation#commit() is called by RDBStore#commitBatchOperation().

Also add javadoc to this getCurrentBatchOperation(), notifying that this returned batch can not be commited.

Copy link
Contributor Author

@amaliujia amaliujia Jan 8, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Glen's idea might be much easier. That's a good option.

I think I will separate it to:

  1. add javadoc to getCurrentBatchOperation() to remind that do not use this batch to commit.
  2. Apply a solution to fix this issue. Created https://issues.apache.org/jira/browse/HDDS-4661 to track it.

Because getCurrentBatchOperation() is not a user-facing API, so only Ozone developer could make such mistakes, and in short term we have code review process to catch those. But I agree we should apply a fix for the longer term.

@amaliujia amaliujia changed the title HDDS-3208. SCM Snapshot Phase 1: Implement Ratis takeSnapshot on SCM HDDS-3208. Implement Ratis takeSnapshot on SCM Jan 7, 2021
@amaliujia
Copy link
Contributor Author

@ChenSammi @runzhiwang @GlenGeng @linyiqun

I have rebased and updated this PR. Now this PR includes the takeSnapshot and loadSnapshot with a test.

@amaliujia
Copy link
Contributor Author

Will fix failing tests.

@amaliujia amaliujia changed the title HDDS-3208. Implement Ratis takeSnapshot on SCM HDDS-3208. Implement Ratis snapshot on SCM Jan 8, 2021
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see this method is used in this PR. Will this be used in follow-up task? If not we could remove this and remove volatile keyword for term/snapshotIndex as well. Because I didn't find the concurrent update for this two variables.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

got it. Will remove this method.

The volatile is a good point. I am also looking for comments to point me out whether there are potential concurrent operations. I will remove volatile if there is no comment about this situation.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The SCMRatisSnapshotInfo is an immutable object after being created. How about make term and snapshotIndex be final and remove updateTermIndex() ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense. Done

Copy link
Contributor

@GlenGeng-awx GlenGeng-awx left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the job!

Better solve following task in this PR, it should be part of the job of load snapshot.
https://issues.apache.org/jira/browse/HDDS-4533

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will it be better to call latestSnapshot = latestTrxInfo.toSnapshotInfo(); here, and remove the setLatestSnapshot () method ?
If caller always has to call flush() and setLatestSnapshot() together, better to merge them to avoid human mistake.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The snapshot info is set also during the initialization. So not only flush() will need to update that information.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about revert the change to process() and change like this ?

applyTransactionFuture.complete(process(request));
transactionBuffer.updateLatestTrxInfo(SCMTransactionInfo.builder()
                   .setCurrentTerm(trx.getLogEntry().getTerm())
                   .setTransactionIndex(trx.getLogEntry().getIndex())
                   .build()));

the process() does not needs to know about the trxInfo

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about move this shouldUpdate() in to SCMTransactionInfo, as a method isEmpty()? We'd better encapulate the magic number 0 and -1 into SCMTransactionInfo .

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Makes sense

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better remove the info in line 133 and replace 156 as

LOG.info("Current Snapshot Index {}, takeSnapshot took {} ms", 
    getLastAppliedTermIndex(), Time.monotonicNow() - startTime);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@amaliujia
Copy link
Contributor Author

Thanks for the job!

Better solve following task in this PR, it should be part of the job of load snapshot.
https://issues.apache.org/jira/browse/HDDS-4533

@GlenGeng I prefer to fix such issue separately so create a PR for it: #1796

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldUpdate() is not very intuitive in the context of SCMTransactionInfo, how about isInitialized() or something like this ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

isInitialized is a good naming.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

DBTransactionBuffer can be fetched by calling SCMHAManager#getDBTransactionBuffer, so this function newPipelineManagerWithMockBuffer is not needed.

// Create PipelineStateManager
StateManager stateManager = PipelineStateManagerV2Impl
    .newBuilder().setPipelineStore(pipelineStore)
    .setRatisServer(scmhaManager.getRatisServer())
    .setNodeManager(nodeManager)
    .setSCMDBTransactionBuffer(scmhaManager.getDBTransactionBuffer())
    .build()

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a very good point.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It should be

if (latestTrxInfo.isInitialized()) {
  updateLastAppliedTermIndex(...)
}

?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually it is !latestTrxInfo.isInitialized().

SCMTransactionInfo latestTrxInfo = buffer.getLatestTrxInfo(); comes from the buffer. The buffer will load the SCMTransactionInfo from the DB. If there was no snapshot taken before (e.g. brand new SCM), there isn't a SCMTransactionInfo from DB thus buffer will create a default one.

So we only need to updateLastAppliedTermIndex when latestTrxInfo is not initialized (thus it is loaded from DB and it's meaningful).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i think, reset of the rocks db batch operation should be made independent of flush. We may/may not require to reinitialise every time we call flush. For example, shutting down the raft server instance may initiate the last snapshot but will not require the batch reinitialisation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with it :-)

In fact we have discussed not to expose batch operation above and I created https://issues.apache.org/jira/browse/HDDS-4661.

As this PR becomes larger and larger so I am planning to address this batch operation related comments in HDDS-4661, including when to init and close it.

How do you feel? Do you agree with my idea?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am ok with addressing in a different PR.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unintended change??

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes :) will undo it.

@amaliujia
Copy link
Contributor Author

Failed UT is the decommission one, which is known to be flaky.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we rename applyTransaction to applyTansactionSerial as this is a serialized operation anyways?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I created https://issues.apache.org/jira/browse/HDDS-4684.

After check the Ratis state machine interface, I can see two functions:

  /**
   * Called for transactions that have been committed to the RAFT log. This step is called
   * sequentially in strict serial order that the transactions have been committed in the log.
   * The SM is expected to do only necessary work, and leave the actual apply operation to the
   * applyTransaction calls that can happen concurrently.
   * @param trx the transaction state including the log entry that has been committed to a quorum
   *            of the raft peers
   * @return The Transaction context.
   */
  TransactionContext applyTransactionSerial(TransactionContext trx);

  /**
   * Apply a committed log entry to the state machine. This method can be called concurrently with
   * the other calls, and there is no guarantee that the calls will be ordered according to the
   * log commit order.
   * @param trx the transaction state including the log entry that has been committed to a quorum
   *            of the raft peers
   */
  CompletableFuture<Message> applyTransaction(TransactionContext trx);

So there is no API as the following

  Message applyTransactionSerial(TransactionContext trx);

Basically a function to apply transaction in strict serial order where it returns a Message.

I am planning to send an email to Ratis community to discuss the intention of only having one applyTransaction that returns Message (CompletableFuture) but can be called concurrently.

We can address this API change in HDDS-4684.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add an annotation of @VisibleForTesting.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The SCMRatisSnapshotInfo is an immutable object after being created. How about make term and snapshotIndex be final and remove updateTermIndex() ?

Copy link
Contributor

@bshashikant bshashikant left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The patch looks ok. The changes discussed will be addressed in subsequent jira.

The transaction buffer flush now can only happen via a ratis snapshot , but in case if ratis is not enabled, there needs to be rocks db sync on every update to db or we need a way to periodically flush the buffer changes to db. This can be done as a part of separate jira.

@bshashikant bshashikant merged commit 3ff677d into apache:HDDS-2823 Jan 20, 2021
@bshashikant
Copy link
Contributor

Thanks @amaliujia for the contribution.

@amaliujia amaliujia deleted the HDDS-3208-new branch January 20, 2021 18:33
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants