-
Notifications
You must be signed in to change notification settings - Fork 1
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
IGNITE-22172 Make ignite-collocation-feature branch stable #102
IGNITE-22172 Make ignite-collocation-feature branch stable #102
Conversation
…dy before returning the primary replica for the placement driver API.
e227c44
to
5a5330a
Compare
modules/core/src/main/java/org/apache/ignite/internal/replicator/ZonePartitionId.java
Outdated
Show resolved
Hide resolved
|
||
return; | ||
} | ||
|
||
if (!success) { | ||
LOG.debug("Lease update invocation failed"); | ||
LOG.warn("Lease update invocation failed."); |
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.
do we need warn here?
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 think a warning is better than just hiding this issue. In ordinary cases, the PD should process all the updates.
Currently, this message might appear because we have a concurrent update of the MC key.
...ent-driver/src/main/java/org/apache/ignite/internal/placementdriver/leases/LeaseTracker.java
Outdated
Show resolved
Hide resolved
@@ -100,7 +104,7 @@ public class Replica { | |||
|
|||
private final ClockService clockService; | |||
|
|||
private final CompletableFuture<Void> waitForActualStateFuture = new CompletableFuture<>(); | |||
private final AtomicReference<CompletableFuture<Void>> waitForActualStateFuture = new AtomicReference<>(); |
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.
Why do we need this change?
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 a better way to avoid starting to wait for the actual state process when several messages are sent.
When one message is in process, the other will wait for the future.
} | ||
|
||
private CompletableFuture<Void> sendPrimaryReplicaChangeToReplicationGroup(long leaseStartTime) { | ||
PrimaryReplicaChangeCommand cmd = REPLICA_MESSAGES_FACTORY.primaryReplicaChangeCommand() | ||
.leaseStartTime(leaseStartTime) | ||
.build(); | ||
|
||
return raftClient.run(cmd); | ||
return raftClient.run(cmd).thenRun(() -> { | ||
if (!waitForActualStateFuture.compareAndSet(null, nullCompletedFuture())) { |
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.
Why do we need this here?
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.
The methodoid is the final phase of the wating for the actual state process.
@@ -351,8 +369,7 @@ private CompletableFuture<Void> waitForActualState(long expirationTime) { | |||
|
|||
return retryOperationUntilSuccess(raftClient::readIndex, e -> currentTimeMillis() > expirationTime, executor) | |||
.orTimeout(timeout, TimeUnit.MILLISECONDS) | |||
.thenCompose(storageIndexTracker::waitFor) | |||
.thenRun(() -> waitForActualStateFuture.complete(null)); | |||
.thenCompose(idx -> storageIndexTracker.waitFor(idx)); |
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.
why have you moved completion of the future from here?
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.
Currently, the process of waiting for the actual state contains two parts:
- waitForActualState
- sendPrimaryReplicaChangeToReplicationGroup
The process is completed when the last stage is completed.
If we look at all places where the waitForActualState is used, you will see the sendPrimaryReplicaChangeToReplicationGroup is always used after.
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java
Outdated
Show resolved
Hide resolved
...replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaAwareLeaseTracker.java
Outdated
Show resolved
Hide resolved
return processWaitReplicaStateMessage(waitReplicaStateMsg) | ||
.thenApply(unused -> EMPTY_REPLICA_RESULT); | ||
} else { | ||
waitForActualStateFuture.get().thenApply(unused -> EMPTY_REPLICA_RESULT); |
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.
What does this line of code do?
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.
This code sends the empty result as a response to WaitReplicaStateMessage. This branch happens if another WaitReplicaStateMessage has already started.
@@ -374,8 +374,7 @@ private void waitForScale(IgniteImpl node, int targetDataNodesCount) throws Inte | |||
assertTrue(IgniteTestUtils.waitForCondition(() -> { | |||
long causalityToken = node.metaStorageManager().appliedRevision(); | |||
|
|||
long msSafeTime = node.metaStorageManager().timestampByRevision(causalityToken).longValue(); | |||
int catalogVersion = node.catalogManager().activeCatalogVersion(msSafeTime); | |||
int catalogVersion = node.catalogManager().latestCatalogVersion(); |
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.
Why have you made this change?
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 was an incorrect test because it capturedd not final catalog state.
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.
Agreed, ms idle safe time is not taken into account while retrieving msSafeTime, my code was buggy
No description provided.