-
Notifications
You must be signed in to change notification settings - Fork 90
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-17834 Fixed. #1168
Ignite-17834 Fixed. #1168
Changes from 2 commits
73fdaa1
357e3f9
6c6eef3
b0e3337
8ab9922
f24f694
ecf4d55
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -415,7 +415,17 @@ public CompletableFuture<T> apply(T r, Throwable e) { | |
throw (RuntimeException) e; | ||
}); // Preserve failed state. | ||
} else { | ||
tx0.enlistResultFuture(fut); | ||
try { | ||
tx0.enlistResultFuture(fut); | ||
} catch (TransactionException enlistResultException) { | ||
return tx0.rollbackAsync().handle((ignored, err) -> { | ||
if (err != null) { | ||
e.addSuppressed(err); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. e is always null here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed. |
||
} | ||
|
||
throw enlistResultException; | ||
}); | ||
} | ||
|
||
return implicit ? tx0.commitAsync().thenApply(ignored -> r) : completedFuture(r); | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,9 @@ | |
|
||
package org.apache.ignite.internal.tx.impl; | ||
|
||
import static java.util.concurrent.CompletableFuture.failedFuture; | ||
import static org.apache.ignite.internal.util.ExceptionUtils.withCause; | ||
import static org.apache.ignite.lang.ErrorGroups.Transactions.TX_ALREADY_FINISHED_ERR; | ||
import static org.apache.ignite.lang.ErrorGroups.Transactions.TX_COMMIT_ERR; | ||
import static org.apache.ignite.lang.ErrorGroups.Transactions.TX_ROLLBACK_ERR; | ||
|
||
|
@@ -54,10 +56,13 @@ public class TransactionImpl implements InternalTransaction { | |
private final TxManager txManager; | ||
|
||
/** Enlisted replication groups: replication group id -> (primary replica node, raft term). */ | ||
private Map<String, IgniteBiTuple<ClusterNode, Long>> enlisted = new ConcurrentSkipListMap<>(); | ||
private final Map<String, IgniteBiTuple<ClusterNode, Long>> enlisted = new ConcurrentSkipListMap<>(); | ||
|
||
/** Enlisted operation futures in this transaction. */ | ||
private volatile List<CompletableFuture<?>> enlistedResults = new ArrayList<>(); | ||
private final List<CompletableFuture<?>> enlistedResults = new ArrayList<>(); | ||
|
||
/** Guard that prevents finishing or enlisting new operations into already finished or finishing transaction. */ | ||
private boolean txInFinishState = false; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Way do not you want to make txInFinishState to AtomicBoolean and get rid of synchronized in the finish method? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Because it's required not only to change txInFinishState atomically but also to populate enlistedResults with new value
|
||
|
||
/** | ||
* The constructor. | ||
|
@@ -137,6 +142,14 @@ public CompletableFuture<Void> rollbackAsync() { | |
* @return The future. | ||
*/ | ||
private CompletableFuture<Void> finish(boolean commit) { | ||
synchronized (this) { | ||
if (txInFinishState) { | ||
return failedFuture(new TransactionException(commit ? TX_COMMIT_ERR : TX_ROLLBACK_ERR, | ||
"Failed to finish already finished transaction txId={" + id + '}')); | ||
} | ||
txInFinishState = true; | ||
} | ||
|
||
// TODO: https://issues.apache.org/jira/browse/IGNITE-17688 Add proper exception handling. | ||
return CompletableFuture | ||
.allOf(enlistedResults.toArray(new CompletableFuture[0])) | ||
|
@@ -181,7 +194,12 @@ private CompletableFuture<Void> finish(boolean commit) { | |
|
||
/** {@inheritDoc} */ | ||
@Override | ||
public void enlistResultFuture(CompletableFuture<?> resultFuture) { | ||
public synchronized void enlistResultFuture(CompletableFuture<?> resultFuture) { | ||
if (txInFinishState) { | ||
throw new TransactionException(TX_ALREADY_FINISHED_ERR, | ||
"Failed to enlist operation into already finished transaction txId={" + id + '}'); | ||
} | ||
|
||
enlistedResults.add(resultFuture); | ||
} | ||
} |
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 are you rolling back the transaction if it is already finished?
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.
Because it's required to cleanup locks and writeIntents for operation that failed to enlistResultFuture.
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.
So that will not happen because the transaction is already finished.
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.
You are right we need an escape path in order to rollback partially finished transaction.