Skip to content
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-20127 Implement delayed replication acks #2520

Closed
wants to merge 35 commits into from

Conversation

ascherbakoff
Copy link
Contributor

@ascherbakoff ascherbakoff commented Aug 30, 2023

Implement delayed replication acks for tx commit.
limitation: doesn't work for sql - needs a separate ticket.
https://issues.apache.org/jira/browse/IGNITE-20127

*
* @return The replication future.
*/
public @Nullable CompletableFuture<?> repFuture() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Just name it as it really is replicationFuture.


if (sender != null) {
// Using strong send here is important to avoid a reordering with a normal response.
clusterNetSvc.messagingService().send(sender, msg0);
Copy link
Contributor

Choose a reason for hiding this comment

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

How will we avoid reordering if the message may be in another thread?

Copy link
Contributor

@sanpwc sanpwc Sep 28, 2023

Choose a reason for hiding this comment

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

Agree. I'd rather add operation id or similar same for both fromPrimary and delayed responses in order to match each other. Such ids should be unique within transaction, so basically it's operation counter or similar. From one point of view that means that it won't be possible to use simple inflights counter, however on the other hand it'll be possible to implement more reliable engine including aforementioned optimizations like single response for single noded groups. WDYT?

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 absence of reordering is guaranteed by current IO implementation. Both messages will use the same channel for sending a message, and a delayed message is send strictly after the initial.

@@ -149,6 +149,8 @@ public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
iterator.forEachRemaining((CommandClosure<? extends WriteCommand> clo) -> {
Command command = clo.command();

// LOG.info("CMD {}", command.getClass().getName());
Copy link
Contributor

Choose a reason for hiding this comment

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

Debugging commnets.

@@ -156,6 +159,8 @@

/** Partition replication listener. */
public class PartitionReplicaListener implements ReplicaListener {
private static final IgniteLogger LOGGER = Loggers.forClass(PartitionReplicaListener.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

We name it LOG, just because we use the same name everywhere.


return txOps;
});
if (txOps.state == TxState.ABORTED || txOps.state == TxState.COMMITED) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Use here TxState#isFinalState

if (txOps == null) {
txOps = new TxCleanupReadyFutureList();
}
var cleanupReadyFut = new CompletableFuture<Void>();
Copy link
Contributor

Choose a reason for hiding this comment

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

I was trying to understand how this future connected with a cleanup procedure, and I realized the future does not connect with the cleanup. I would rename this variable to 'replicationReadyFut'.

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 future is used to prevent races between inflight ops and commit/abort, as it was before my patch. No need to change anything here.

return cmd.txId();
});

// TODO error handling similar to replication group.
Copy link
Contributor

Choose a reason for hiding this comment

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

All TODO comments should a ticket number.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No longer actual, removed.

*/
private CompletableFuture<Object> applyUpdateCommand(UpdateCommand cmd) {
private CompletableFuture<CompletableFuture<?>> applyUpdateCommand(UpdateCommand cmd) {
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 it necessary to use this unclear logic. This method returns either a result or a replication future. A future wrapper is never required.

Copy link
Contributor

Choose a reason for hiding this comment

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

Agree. Proposed API is overcomplicated.

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 signature is required to implement correct chaining both for implicit and explicit tx flows.
The result of a call either the immediate result with a delayed replication future, or immediate replication future.
Actually this is the simplest way to implement it.
Can you propose simpler variant, which is also correct ?

@@ -1853,36 +1902,68 @@ private CompletableFuture<Object> applyUpdateCommand(UpdateCommand cmd) {
* @param cmd UpdateAll command.
* @return Raft future, see {@link #applyCmdWithExceptionHandling(Command)}.
*/
private CompletableFuture<Object> applyUpdateAllCommand(UpdateAllCommand cmd) {
private CompletableFuture<CompletableFuture<?>> applyUpdateAllCommand(UpdateAllCommand cmd, boolean skipDelayedAck) {
Copy link
Contributor

Choose a reason for hiding this comment

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

The same is before. The future wrapper is not need here.

@@ -106,6 +117,8 @@
* Storage of table rows.
*/
public class InternalTableImpl implements InternalTable {
private static final IgniteLogger LOGGER = Loggers.forClass(InternalTableImpl.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

We name it LOG, just because we use the same name everywhere.

@@ -57,7 +65,12 @@
*
* <p>Uses 2PC for atomic commitment and 2PL for concurrency control.
*/
public class TxManagerImpl implements TxManager {
public class TxManagerImpl implements TxManager, NetworkMessageHandler {
private static final IgniteLogger LOGGER = Loggers.forClass(TxManagerImpl.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

LOG

return replicaService.invoke(recipientNode, req)
// TODO: IGNITE-20033 TestOnly code, let's consider using Txn state map instead of states.
.thenRun(() -> changeState(txId, PENDING, commit ? COMMITED : ABORTED));
Function<Void, CompletableFuture<Void>> clo = ignored -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do not use Supplier<CompletableFuture<Void>>?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Because Supplier can not be used in completableFuture.thenCompose

@Override
public void removeInflight(UUID txId) {
TxContext tuple = txCtxMap.compute(txId, (uuid, ctx) -> {
assert ctx != null;
Copy link
Contributor

Choose a reason for hiding this comment

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

Add a check that the counter of the flight operations is greater than 0.

.thenRun(() -> changeState(txId, PENDING, commit ? COMMITED : ABORTED));
};

if (!commit) {
Copy link
Contributor

Choose a reason for hiding this comment

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

The logic for rollback and commit operations are duplicated. You can just make this on the commit flag:

// All inflights have been completed before the finish.
if (commit && tuple.inflights == 0) {
     tuple.waitRepFut.complete(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.

I've removed code duplication.

@@ -57,7 +65,12 @@
*
* <p>Uses 2PC for atomic commitment and 2PL for concurrency control.
*/
public class TxManagerImpl implements TxManager {
public class TxManagerImpl implements TxManager, NetworkMessageHandler {
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 where TxManager is used as a NetworkMessageHandler. Also, I do not understand why it is necessary to calculate inflight operations for the algorithm.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Here org.apache.ignite.internal.tx.impl.TxManagerImpl#start
Inflight calculation is required for safe committing a transaction, see ticket description.

if (v == before) {
return after;
} else {
return v;
}
});

assert computeResult == after : "Unable to change transaction state, expected = [" + before + "],"
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 not sure if it is right to delete this asset because the redundant state changes invoked are not expected.

Copy link
Contributor

Choose a reason for hiding this comment

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

Agree, assert seems useful.

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 assertion is no longer in the master.

@@ -42,6 +42,7 @@ public interface MessagingService {
* <li>Messages send to same receiver will be delivered in the same order as they were sent;</li>
* <li>If a message N has been successfully delivered to a member implies that all messages to same receiver
* preceding N have also been successfully delivered.</li>
* <li>A message is delivered only once</li>
Copy link
Contributor

Choose a reason for hiding this comment

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

Full stop is missing in the end.

@@ -115,6 +115,8 @@ public DefaultMessagingService(
this.marshaller = marshaller;

this.outboundExecutor = Executors.newSingleThreadExecutor(NamedThreadFactory.create(nodeName, "MessagingService-outbound-", LOG));
// TODO asch the implementation of delayed acks relies on absence of reordering on subsequent messages delivery.
Copy link
Contributor

Choose a reason for hiding this comment

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

Why?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Because if it is not held the commit can be called with some write operation in progress.
But we need to enforce the following invariant on commit - all inflight operations are finished and all corresponding replication is completed.
Actually, looks like we can relax this invariant to "all corresponding replication is completed", because it implies that all updates on primary replica is also applied.

LOG.debug("Sending delayed response for replica request [request={}]", request);

if (ex == null) {
msg0 = prepareReplicaResponse(finalSendTimestamp, res0);
Copy link
Contributor

Choose a reason for hiding this comment

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

What kind of result you expect in repFut (res0). I believe that it is void in any case because it basically tells us that replication is finished or throws an exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is not always void. In delayed ack response txId is set as a result. See applyUpdateCommand

@@ -261,6 +261,35 @@ private void onReplicaMessageReceived(NetworkMessage message, String senderConsi
}
}

if (res.repFuture() != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

How do we handle single noded raft groups? I didn't check the code below, maybe it's already implemented. But for now, my questions/expectations are following:

  • In case of single noded raft group and if primary is collocated with that single node, we should skip primary storage updates, and just send the command for the replication, synchronously await the result and return the future with completed res.repFuture and thus send single response to the client side.
  • Otherwise use common logic with two responses.
  • In all cases we should check whether repFuture is completed when we are ready to send response from the primary and send only one response if it is.

BTW if items 1 and 2 aren't implemented yet, I'd rather do it in a separate ticket.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No special handling required, it just works as is. But you are correct - it may be additionally optimized. I suggest a separate ticket.

msg0 = prepareReplicaErrorResponse(finalSendTimestamp, ex);
}

ClusterNode sender = clusterNetSvc.topologyService().getByConsistentId(senderConsistentId);
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd rather add overloaded version of send with recipientConsistentId as we discussed with Ivan and just skip the clusterNodeResolver part.
We already have one with channelType param
CompletableFuture<Void> send(String recipientConsistentId, ChannelType channelType, NetworkMessage msg);

CompletableFuture<Void> fut = accounts.recordView().upsertAsync(tx, makeValue(1, 100.));
// Update must complete now despite the blocked replication protocol.
assertTrue(IgniteTestUtils.waitForCondition(fut::isDone, 5_000), "The update future is not completed within timeout");

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you have an idea of how to check that transaction won't finish until unblocking?

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 haven't


assertFalse(replicaWriteFut.isDone());
assertTrue(replicaWriteFut.isDone());
Copy link
Contributor

Choose a reason for hiding this comment

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

Why?

@@ -1541,7 +1580,7 @@ public void run() {
}

while (!stop.get() && firstErr.get() == null) {
InternalTransaction tx = txManager(accounts).begin();
InternalTransaction tx = clientTxManager().begin();
Copy link
Contributor

Choose a reason for hiding this comment

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

Is an intended change? I mean that, despite the fact that we still have txManager() we want use client ones, right?

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 it is. Previously a transaction has been started using server txManager, instead of an client.

@@ -33,7 +33,6 @@ public interface LockManager {
* @param lockKey The key.
* @param lockMode Lock mode, for example shared, exclusive, intention-shared etc.
* @return The future with gained lock that will be completed when a lock is successfully acquired.
* @throws LockException When a lock can't be taken due to possible deadlock.
Copy link
Contributor

Choose a reason for hiding this comment

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

Why?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Because this exception is never thrown.

if (res instanceof ReplicaResult) {
return (ReplicaResult) res;
} else {
return new ReplicaResult(res, null);
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd rather introduce overloaded version with one param.

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 will not work, because ReplicaResult represents two results. See my previous comment about it.

@@ -1561,6 +1591,7 @@ private CompletableFuture<Object> processMultiEntryAction(ReadWriteMultiRowRepli

int i = 0;

// TODO sort keys ?
Copy link
Contributor

Choose a reason for hiding this comment

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

Why?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not actual

}

return updateAllCommand(request, rowIdsToDelete)
.thenCompose(this::applyUpdateAllCommand)
.thenApply(ignored -> result);
.thenCompose(cmd -> applyUpdateAllCommand(cmd, request.skipDelayedAck()))
Copy link
Contributor

Choose a reason for hiding this comment

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

Same as above. Feel free to answer in one topic only. Why we skipDelayedAck in case of _ALL?

CompletableFuture<Object> raftFut = rowIdsToDelete.isEmpty() ? completedFuture(null)
: updateAllCommand(request, rowIdsToDelete).thenCompose(this::applyUpdateAllCommand);
if (rowIdsToDelete.isEmpty()) {
return completedFuture(new ReplicaResult(result, null));
Copy link
Contributor

Choose a reason for hiding this comment

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

That's a tricky part. We still may want to update safeTime even if there are no rows to delete, otherwise we will need to await idle safe time propagation. We may cover this with separate optimization ticket, but for now I'd prefer calling applyUpdateAllCommand even for empty rows for the simplicity.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure if this is actual after merging.

@@ -1853,36 +1902,68 @@ private CompletableFuture<Object> applyUpdateCommand(UpdateCommand cmd) {
* @param cmd UpdateAll command.
* @return Raft future, see {@link #applyCmdWithExceptionHandling(Command)}.
*/
private CompletableFuture<Object> applyUpdateAllCommand(UpdateAllCommand cmd) {
private CompletableFuture<CompletableFuture<?>> applyUpdateAllCommand(UpdateAllCommand cmd, boolean skipDelayedAck) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Code seems reasonably complicated. It worth adding comments explaining the flow.

CompletableFuture<Object> fut = applyCmdWithExceptionHandling(cmd).thenApply(res -> {
// Currently result is always null on a successfull execution of a replication command.
// This check guaranties the result will never be lost.
assert res == null : "Replication result is lost";
Copy link
Contributor

Choose a reason for hiding this comment

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

Well, from my point of view it will always be null, because basically it's void.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

But the method signature says otherwise. The result can appear later, so I've added some protection.

* @param tx The transaction id.
* @return {@code True} if the inflight was registered. The update must be failed on false.
*/
boolean addInflight(UUID txId);
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd say that add/removeInflight should be a part of InternalTransaction and not TxManager.
Besides that javadoc requires clarification. And what's more important sql currently totally miss the thing.

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 disagree. It makes sense only for RW transactions, so it conflicts with RO implementation.

/**
* Registers the infligh update for a transaction.
*
* @param tx The transaction id.
Copy link
Contributor

Choose a reason for hiding this comment

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

txId not tx, but as mentioned below probably not relevant because I believe we should move given method to InternalTransaction.

) {
// LOGGER.info("enlistWithRetry {} {}", partId, tx.id().toString());
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove.

@@ -486,13 +476,54 @@ private <R> CompletableFuture<R> enlistWithRetry(
return result;
}

private <R> CompletableFuture<R> trackingInvoke(
Copy link
Contributor

Choose a reason for hiding this comment

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

Given method deserves javadoc.

|| request instanceof MultipleRowReplicaRequest && ((MultipleRowReplicaRequest) request).requestType() != RW_GET_ALL;

if (write && !full) {
// Track only write requests from explicit transactions.
Copy link
Contributor

Choose a reason for hiding this comment

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

Why we track only write requests. AFAIK we should fail reading operations within finished transactions as well as modifications ones. WDYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Tracking is required only for wait replication on commit purposes.

assert noOpChecker != null;

// Remove inflight if no replication was scheduled, otherwise inflight will be removed by delayed response.
if (noOpChecker.test(res, request)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

noOpCheck naming seems confusing to me.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Renamed to noWriteCheck

@@ -509,16 +540,16 @@ private <T> CompletableFuture<T> postEnlist(CompletableFuture<T> fut, boolean au
if (e != null) {
RuntimeException e0 = wrapReplicationException(e);

//LOGGER.info("DBG: before auto rollback {}", tx0.id());
Copy link
Contributor

Choose a reason for hiding this comment

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

Please remove, here and below.

AtomicReference<CompletableFuture<Void>> ref = new AtomicReference<>();
TxContext tuple = txCtxMap.compute(txId, (uuid, tuple0) -> {
if (tuple0 == null) {
tuple0 = new TxContext(); // No writes enlisted.
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 be consistent with the naming. enlistment and in-flight operations have different meaning. It's minor of course.

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 naming is correct here - inflighs are caused by enlisted writes.

tuple.waitRepFut.complete(null);
}

return tuple.waitRepFut.thenCompose(clo).handle((ignored, err) -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

In order to be on the same page. We do wait for all in-flights to complete even in case of rollback, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No we do not.

…nto ignite-20127

� Conflicts:
�	modules/client/src/test/java/org/apache/ignite/client/fakes/FakeTxManager.java
�	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java
�	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
�	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/listener/ReplicaListener.java
�	modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
�	modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
�	modules/table/src/integrationTest/java/org/apache/ignite/distributed/ReplicaUnavailableTest.java
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replication/request/ReadWriteMultiRowReplicaRequest.java
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/TxLocalTest.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
�	modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
�	modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
�	modules/transactions/src/main/java/org/apache/ignite/internal/tx/InternalTransaction.java
�	modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
�	modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
…nto ignite-20127

� Conflicts:
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/TxLocalTest.java
�	modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/ReadWriteTransactionImpl.java
�	modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxManagerImpl.java
…nto ignite-20127

� Conflicts:
�	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
…nto ignite-20127

� Conflicts:
�	modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java
�	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
�	modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
@ascherbakoff ascherbakoff changed the title IGNITE-20127 IGNITE-20127 Implement delayed replication acks Oct 6, 2023
@asfgit asfgit closed this in 82c7459 Oct 7, 2023
Lozmanov pushed a commit to Lozmanov/ignite-3 that referenced this pull request Nov 6, 2023
* IGNITE-19425 Use data nodes from DistributionZoneManager instead of BaselineManager#nodes and remove BaselineManager (apache#2605)

* IGNITE-20127 Implement delayed replication acks - Fixes apache#2520.

Signed-off-by: Alexey Scherbakov <alexey.scherbakoff@gmail.com>

* IGNITE-20330 Create an abstraction for building indexes (apache#2631)

* IGNITE-20513 Use NativeType as the type of the system view column (apache#2668)

* IGNITE-20590 Remove TxManager.beginImplicit() (apache#2669)

* IGNITE-20588 Use consistent schema when converting data in KV/record views (apache#2667)

* IGNITE-20342 Rollback transaction for SQL execution issues (apache#2611)

* IGNITE-20445 Clean up write intents for RW transaction on replication… (apache#2657)

* IGNITE-20607 Move NativeType to core (apache#2670)

* IGNITE-20484 NPE when some operation occurs when the primary replica … (apache#2672)

* IGNITE-20442 Sql. Extend grammar with transaction related statements. (apache#2663)

* IGNITE-20620 Add index availability command to catalog (apache#2680)

* IGNITE-20521 Split Security API from ignite-security module (apache#2662)

* IGNITE-20621 Add a replication group index build completion listener to IndexBuilder (apache#2682)

* IGNITE-19226 Fetch table schema by timestamp (apache#2648)

* IGNITE-19325 Unmute test MultiActorPlacementDriverTest#prolongAfterActiveActorChanger (apache#2677)

* IGNITE-20625 Fix ODBC metadata reading error (apache#2687)

* IGNITE-20457 Verify commitTimestamp against enlisted partitions expiration timestamps (apache#2658)

* IGNITE-20635 Cleanup code wrt IGNITE-18733 mentions (apache#2686)

* IGNITE-20317 Return metastorage invokes for zones changes in handlers, immediately recalculate data nodes when scale up/down is immediate. (apache#2685)

* IGNITE-19276 Implement a mechanism to build indices distributively (apache#2676)

* IGNITE-20567 Move the 'enabled' flag from the authentication configuration to security (apache#2665)

* IGNITE-20578 Implement scan over system view (apache#2678)

* IGNITE-20387: Remap most exceptions to SqlExceptions for SQL API (apache#2613)

* IGNITE-19217 Implement foreign keys query for ODBC  (apache#2692)

* IGNITE-20512 Remove port range from HTTP server (apache#2673)

* Remove port range from HTTP server configuration
* Update junit5 version to avoid jar hell.

* IGNITE-20657 At the checkpoint ArrayIndexOutOfBoundsException (apache#2696)

* IGNITE-20444 Sql. Add restrictions for execution tx related statements with single statement mode (apache#2683)

* IGNITE-20116 Linearize storage updates with safeTime adjustment rules (apache#2689)

* IGNITE-20636 Add to the MakeIndexAvailableCommand the ability to use only the indexId (apache#2691)

* IGNITE-19219  Implement primary keys query in ODBC (apache#2699)

* IGNITE-20366 testBatchReadPutConcurrently failed (apache#2694)

* IGNITE-20435 Preserve key order in batch opperations (deleteAll, deleteAllExact, insertAll) (apache#2664)

* IGNITE-20385 Sql. Fixed handling of NodeLeftException (apache#2622)

* IGNITE-20672 Broke compilation (apache#2705)

* IGNITE-20530 Start building indexes for write-only indexes (apache#2704)

* IGNITE-20659 Placement driver do not create a lease (apache#2700)

* IGNITE-20668 Increase wait after a DDL to account for idle safe-time propagation period (apache#2703)

* IGNITE-20522 Create the default user 'Ignite' (apache#2684)

* Ignite/ignite default user is added to the configuration if the user did not specify one
* security.authentication.enabled -> security.enabled
* We didn't have the ability to set dynamic defaults in the current implementation of configuration before this patch. Moreover, we don't have the ability to set a default value for NamedConfigValue at all. Now you can add the code to the configuration module if you want to add some defaults based on user-provided values or other external conditions.

* IGNITE-20616 ensureReplicaIsPrimary should use getPrimaryReplica instead of awaitPrimaryReplica (apache#2701)

* IGNITE-20671 Sql. Fixed ItSqlApiTest#ddl test (apache#2708)

* IGNITE-20677 Sql. Improve error reporting in assertThrowsSqlException (apache#2707)

* IGNITE-20600 Sql. Fix a message of an error, which occurs while updating primary key column (apache#2695)

* IGNITE-20478 Sql. Get rid of UNSPECIFIED_VALUE_PLACEHOLDER (apache#2671)

* IGNITE-20430 Got rid of unused set and fixed replica waiters removal (apache#2604)

* IGNITE-20454 Sql. Added a callback that is notified when data prefetching is complete (apache#2674)

* IGNITE-20002 Implement durable unlock on primary partition re-election (apache#2697)

* IGNITE-20630 Select only available nodes for deployment unit download (apache#2713)

* IGNITE-20693 Fixed NPE in placement driver actor on deactivation (apache#2718)

* IGNITE-20695 Cleanup resource (apache#2723)

* IGNITE-20545 Improve logging in AbstractRpcTest (apache#2717)

* IGNITE-20395 Clean up write intents for RW transaction on primary (apache#2679)

* IGNITE-20699 Decrease idle safe time propagation period in tests (apache#2730)

* IGNTIE-20629 Exclude ODBC build from assemble pipeline by default (apache#2706)

* IGNITE-20706 Rename CatalogSchemaManager to SchemaManager (apache#2733)

* IGNITE-20702 Fix NPE in ReplicaManager.onReplicaMessageReceived (apache#2731)

* IGNITE-20704 Add methods to fsync files and directories (apache#2735)

* IGNITE-20703 LeaseUpdaterTest causes a NullPointerException (apache#2732)

* IGNITE-20599 Implement 'NOT_TOMBSTONE' operation in the meta storage dsl. (apache#2722)

* IGNITE-20042 Check table existence before executing each operation in an RW transaction (apache#2721)

* IGNITE-20576 Fix testGetReturningTupleWithUnknownSchemaRequestsNewSchema (apache#2738)

* IGNITE-20359 Expose storage profiles as a node attribute (apache#2711)

* IGNITE-20439 Sql. Support multiple schemas in CatalogSqlSchemaManager (apache#2719)

* IGNITE-20311 Sql. Fix behaviour of ROUND function (apache#2690)

* IGNITE-20684 Use proper sync method (apache#2715)

* IGNITE-20670 Extract deployment code integration test to related module (apache#2714)

* IGNITE-20304 Sql. Documentation for system views module (apache#2726)

* IGNITE-20644 Java thin: Fix error detection in doSchemaOutInOpAsync (apache#2739)

Fix well-known error detection in `ClientTable` when `sendServerExceptionStackTraceToClient` is enabled: `unwrapRootCause` did not work, the important exception can be on any level.

* IGNITE-20498 Fix potential catalog version order violations (apache#2734)

* IGNITE-20560 Remove the field out of sync with the map (apache#2660)

* IGNITE-20099 Update okhttp version (apache#2746)

* IGNITE-20720 Move ClusterPerTestIntegrationTest and ClusterPerClassIntegrationTest to Test Fixtures (apache#2744)

* IGNITE-20618 Sql. Degradation of SELECT operations performance over time (apache#2728)

* IGNITE-20726: Fix incorrect link that mentions a closed ticket. (apache#2745)

* IGNITE-20727 Pass schema version in each read/write ReplicaRequest (apache#2747)

* IGNITE-20561 Change condition for DistributionZonesUtil#triggerKeyConditionForZonesChanges to solve inconsistency issues (apache#2743)

* IGNITE-20734 Fixed a link to cli doc (apache#2750)

* IGNITE-20739 fix compilation after merge of IGNITE-20561 (apache#2753)

* IGNITE-20624 Fix race between getting logical topology and mapping fragments (apache#2710)

---------

Signed-off-by: Alexey Scherbakov <alexey.scherbakoff@gmail.com>
Co-authored-by: Mirza Aliev <alievmirza@gmail.com>
Co-authored-by: Alexey Scherbakov <alexey.scherbakoff@gmail.com>
Co-authored-by: Kirill Tkalenko <tkalkirill@yandex.ru>
Co-authored-by: korlov42 <korlov@gridgain.com>
Co-authored-by: Roman Puchkovskiy <roman.puchkovskiy@gmail.com>
Co-authored-by: Max Zhuravkov <shhwwa@gmail.com>
Co-authored-by: Cyrill <cyrill.sizov@gmail.com>
Co-authored-by: Ivan Gagarkin <gagarkin.iiu@gmail.com>
Co-authored-by: Vladislav Pyatkov <vldpyatkov@gmail.com>
Co-authored-by: Igor Sapego <isapego@apache.org>
Co-authored-by: Alexander Lapin <lapin1702@gmail.com>
Co-authored-by: ygerzhedovich <41903880+ygerzhedovich@users.noreply.github.com>
Co-authored-by: Aleksandr Pakhomov <apkhmv@gmail.com>
Co-authored-by: Pavel Pereslegin <xxtern@gmail.com>
Co-authored-by: Evgeniy Stanilovskiy <stanilovsky@gmail.com>
Co-authored-by: Denis Chudov <moonglloom@gmail.com>
Co-authored-by: Mikhail <Pochatkin@users.noreply.github.com>
Co-authored-by: Vadim Pakhnushev <8614891+valepakh@users.noreply.github.com>
Co-authored-by: Alexander Polovtcev <alex.polovtcev@gmail.com>
Co-authored-by: Pavel Tupitsyn <ptupitsyn@apache.org>
Co-authored-by: Andrew V. Mashenkov <AMashenkov@users.noreply.github.com>
Co-authored-by: IgGusev <igusev@gridgain.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants