-
Notifications
You must be signed in to change notification settings - Fork 594
HDDS-8508. SCMHATransactionBuffer flush based on time #4683
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
| .setPeriodicalTask(() -> { | ||
| monitorTask.run(); | ||
| }).build(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| .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); |
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
Xushaohong
left a comment
There was a problem hiding this 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.
@Xushaohong Its updated |
Xushaohong
left a comment
There was a problem hiding this 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"; |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@Xushaohong Updated with comment
szetszwo
left a comment
There was a problem hiding this 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++; |
There was a problem hiding this comment.
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.
| if (null == transactionBuffer.getLatestSnapshot()) { | ||
| transactionBuffer.setLatestSnapshot(lastSnapshot); | ||
| } |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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.
| 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."); |
There was a problem hiding this comment.
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()); |
There was a problem hiding this comment.
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 { |
There was a problem hiding this comment.
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 @@ | |||
| /** | |||
There was a problem hiding this comment.
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.
| * 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. |
There was a problem hiding this comment.
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
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java
Show resolved
Hide resolved
| private SCMRatisServer server; | ||
| private SCMHADBTransactionBuffer transactionBuffer; | ||
| private long flushInterval = 0; |
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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?
szetszwo
left a comment
There was a problem hiding this 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
| // the config will transfer value to ratis config | ||
| // raft.server.snapshot.auto.trigger.threshold |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use javadoc comments.
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHAManagerImpl.java
Show resolved
Hide resolved
| TimeUnit.MILLISECONDS); | ||
| SCMHATransactionBufferMonitorTask monitorTask | ||
| = new SCMHATransactionBufferMonitorTask( | ||
| (SCMHADBTransactionBuffer) transactionBuffer, ratisServer, interval); |
There was a problem hiding this comment.
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; |
There was a problem hiding this comment.
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); |
There was a problem hiding this comment.
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") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Remove @SuppressWarnings
| if (null == transactionBuffer.getLatestSnapshot()) { | ||
| transactionBuffer.setLatestSnapshot(lastSnapshot); | ||
| } |
There was a problem hiding this comment.
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(); |
There was a problem hiding this comment.
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).
szetszwo
left a comment
There was a problem hiding this 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
szetszwo
left a comment
There was a problem hiding this 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
|
@szetszwo Thanks for suggestion for fixing latestSnapshot, I have fixed all comments. Plz review. |
szetszwo
left a comment
There was a problem hiding this 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.
What changes were proposed in this pull request?
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.