Skip to content

Conversation

@sumitagrawl
Copy link
Contributor

What changes were proposed in this pull request?

  • Timer based flush is added using ratis SnapshotManagement

What is the link to the Apache JIRA

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

How was this patch tested?

UT cases and integration test case.

Comment on lines 115 to 117
.setPeriodicalTask(() -> {
monitorTask.run();
}).build();
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
.setPeriodicalTask(() -> {
monitorTask.run();
}).build();
.setPeriodicalTask(monitorTask)
.build();

Could be simpler

LOG.debug("Running TransactionFlushTask");
// set latest snapshot to null for force snapshot
// the value will be reset again when snapshot is taken
transactionBuffer.setLatestSnapshot(null);
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this necessary? What will happen if we do not set null

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@Xushaohong Thanks for review, above is done for full snapshot as,

  • Ratis will try to get diff of current applied Index and latest snapshot Index, with minGapValue (default value is 1024), if its less than do not take snapshot
  • But since this is force snapshot, and setting LatestSnapshot to null, its value is treated as "0", so diff after sometime crosses 1024 value and snapshot is taken.
  • Further reset is not required with old value as when taking snapshot, it will set with latest applied index. (I will add if value is not set during snapshot due to any failure for this case).

Note: At least 1024 transaction is required in this case also, as this limitation should be ok after discussion with Nicholas,

@szetszwo Please share if this logic is fine as discussed.

Copy link
Contributor

@Xushaohong Xushaohong May 16, 2023

Choose a reason for hiding this comment

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

I see. In Ratis, the gap needs this to bypass the limitation. Thx for the explanation.

Copy link
Contributor

@Xushaohong Xushaohong left a comment

Choose a reason for hiding this comment

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

Thx @sumitagrawl for the work. The idea of adding a timer to timely trigger the Ratis snapshot mostly looks fine to me. Left a few comments.

Besides, we need to reset txFlushPending in SCMHADBTransactionBufferImpl#init. When a snapshot is loaded and reinited, the txFlushPending should be reset also.

@sumitagrawl sumitagrawl requested a review from Xushaohong May 10, 2023 14:26
@sumitagrawl
Copy link
Contributor Author

Thx @sumitagrawl for the work. The idea of adding a timer to timely trigger the Ratis snapshot mostly looks fine to me. Left a few comments.

Besides, we need to reset txFlushPending in SCMHADBTransactionBufferImpl#init. When a snapshot is loaded and reinited, the txFlushPending should be reset also.

@Xushaohong Its updated

Copy link
Contributor

@Xushaohong Xushaohong left a comment

Choose a reason for hiding this comment

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

Sorry for the late reply.
The change looks good to me! Thx @sumitagrawl.
See the latest suggestion, could be fine without that.

Hi @szetszwo, could you help take a look at this PR?


public static final String OZONE_SCM_HA_RATIS_SNAPSHOT_GAP
= "raft.server.snapshot.creation.gap";
= "ozone.scm.ha.ratis.server.snapshot.creation.gap";
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you refine the name and add some comments to help with the comprehension for the other reader? It might be confusing with the AutoTriggerThreshold as it is only used when RATIS takeSnapshotAsync by manual.
How about ozone.scm.ha.ratis.snapshot.creation.auto.threshold and ozone.scm.ha.ratis.snapshot.creation.manual.gap respectively?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@Xushaohong Thanks for review,
"raft.server.snapshot.creation.gap" -- This is configuration provided by ratis to trigger / take snapshot.
"ozone.scm.ha.ratis.server.snapshot.creation.gap" -- This configuration is provided at ozone mapping to ratis, which can be updated. And is applicable for both manual snapshot and raits automatic snapshot.

So adding manual is not required.

Copy link
Contributor

Choose a reason for hiding this comment

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

Hi @sumitagrawl
Actually, I mean these two: ScmConfigKeys.OZONE_SCM_HA_RATIS_SNAPSHOT_THRESHOLD and OZONE_SCM_HA_RATIS_SNAPSHOT_GAP.
They are both SCM side conf, but their similar names could be confusing for those not too familiar.
It might be better to add some comments to tell the difference or rename them.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@Xushaohong Updated with comment

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@sumitagrawl , thanks a lot for working on this! Please see the comments inlined.

Table<KEY, VALUE> table, KEY key, VALUE value) throws IOException {
rwLock.readLock().lock();
try {
txFlushPending++;
Copy link
Contributor

Choose a reason for hiding this comment

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

It need AtomicLong or writeLock.

Comment on lines 63 to 65
if (null == transactionBuffer.getLatestSnapshot()) {
transactionBuffer.setLatestSnapshot(lastSnapshot);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

getLatestSnapshot and setLatestSnapshot are not thread safe.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated ... added lock while update or get snapshot

Copy link
Contributor

Choose a reason for hiding this comment

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

It is still not working get and set are two different calls. We need to use AtomicReference.compareAndSet.

Comment on lines 235 to 240
final long requestTimeout = ozoneConf.getTimeDuration(
ScmConfigKeys.OZONE_SCM_HA_RATIS_REQUEST_TIMEOUT,
ScmConfigKeys.OZONE_SCM_HA_RATIS_REQUEST_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
Preconditions.checkArgument(requestTimeout > 1000L,
"Ratis request timeout cannot be less than 1000ms.");
Copy link
Contributor

Choose a reason for hiding this comment

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

Declare requestTimeout as a field and move the check to the constructor.

nextCallId(), requestTimeout);
final RaftClientReply raftClientReply = server.snapshotManagement(req);
if (!raftClientReply.isSuccess()) {
LOG.info("Snapshot request failed", raftClientReply.getException());
Copy link
Contributor

Choose a reason for hiding this comment

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

Use LOG.warn.

}

@Override
public boolean doSnapshotRequest() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's call it triggerSnapshot.

@@ -0,0 +1,69 @@
/**
Copy link
Contributor

Choose a reason for hiding this comment

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

Please don't use javadoc for license.

Comment on lines 2 to 15
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
Copy link
Contributor

Choose a reason for hiding this comment

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

Please copy a well formatted license header; see https://www.apache.org/legal/src-headers.html

Comment on lines 31 to 33
private SCMRatisServer server;
private SCMHADBTransactionBuffer transactionBuffer;
private long flushInterval = 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

Add final.

TimeUnit.MILLISECONDS);
SCMHATransactionBufferMonitorTask monitorTask
= new SCMHATransactionBufferMonitorTask(
(SCMHADBTransactionBuffer) transactionBuffer, ratisServer, interval);
Copy link
Contributor

Choose a reason for hiding this comment

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

When HA is disabled, the cast will fail.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

There is a check before calling this method in start(), if ratisServer is null, return. So this flow will not hit when HA is disabled.

Copy link
Contributor

Choose a reason for hiding this comment

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

I see. Could you move the method to right below the start() method?

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@sumitagrawl , thanks for the update!

We need to declare latestSnapshot as AtomicReference<SnapshotInfo>; see https://issues.apache.org/jira/secure/attachment/13058517/4683_review.patch

Comment on lines 559 to 560
// the config will transfer value to ratis config
// raft.server.snapshot.auto.trigger.threshold
Copy link
Contributor

Choose a reason for hiding this comment

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

Use javadoc comments.

TimeUnit.MILLISECONDS);
SCMHATransactionBufferMonitorTask monitorTask
= new SCMHATransactionBufferMonitorTask(
(SCMHADBTransactionBuffer) transactionBuffer, ratisServer, interval);
Copy link
Contributor

Choose a reason for hiding this comment

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

I see. Could you move the method to right below the start() method?

package org.apache.hadoop.hdds.scm.ha;

import com.google.common.base.Preconditions;
import java.util.concurrent.atomic.AtomicLong;
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you move it right above to import java.util.concurrent.locks.ReentrantReadWriteLock;?

private BatchOperation currentBatchOperation;
private TransactionInfo latestTrxInfo;
private SnapshotInfo latestSnapshot;
private AtomicLong txFlushPending = new AtomicLong(0);
Copy link
Contributor

Choose a reason for hiding this comment

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

Add final.

* SCMService related variables.
*/

@SuppressWarnings("parameternumber")
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove @SuppressWarnings

Comment on lines 63 to 65
if (null == transactionBuffer.getLatestSnapshot()) {
transactionBuffer.setLatestSnapshot(lastSnapshot);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

It is still not working get and set are two different calls. We need to use AtomicReference.compareAndSet.

LOG.debug("Running TransactionFlushTask");
// set latest snapshot to null for force snapshot
// the value will be reset again when snapshot is taken
SnapshotInfo lastSnapshot = transactionBuffer.getLatestSnapshot();
Copy link
Contributor

Choose a reason for hiding this comment

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

We should use AtomicReference.getAndSet(null).

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@sumitagrawl , thanks for the update!

We need to declare latestSnapshot as AtomicReference<SnapshotInfo>; see https://issues.apache.org/jira/secure/attachment/13058517/4683_review.patch

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@sumitagrawl , thanks for the update!

We need to declare latestSnapshot as AtomicReference<SnapshotInfo>; see https://issues.apache.org/jira/secure/attachment/13058517/4683_review.patch

@sumitagrawl sumitagrawl requested a review from szetszwo May 25, 2023 12:12
@sumitagrawl
Copy link
Contributor Author

@szetszwo Thanks for suggestion for fixing latestSnapshot, I have fixed all comments. Plz review.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

+1 the change looks good.

@szetszwo szetszwo merged commit 72bce3d into apache:master May 25, 2023
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.

3 participants