[
https://issues.apache.org/jira/browse/IGNITE-27812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Vyacheslav Koptilin updated IGNITE-27812:
-----------------------------------------
Description:
{noformat}
java.lang.AssertionError: Metric name = PendingWriteIntentsExpected:
is <0L> but: was <1L>
Expected :is <0L>
Actual :<1L>
at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20)
at
org.apache.ignite.internal.tx.ItTransactionMetricsTest.testMetricValues(ItTransactionMetricsTest.java:123)
at
org.apache.ignite.internal.tx.ItTransactionMetricsTest.testRollbackTransactionOnLeaseExpiration(ItTransactionMetricsTest.java:368)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
at java.base/java.util.ArrayList.forEach(ArrayList.java:1541) {noformat}
It seems to me, that the root cause is that the client's future might be
completed before we initiate the transaction clean-up process that should
release locks and switch write intents. See
`TransactionInflights.preformFinish()`:
{code:title=TransactionInflights.preformFinish()|language=java}
CompletableFuture<Void> performFinish(boolean commit, Function<Boolean,
CompletableFuture<Void>> finishAction) {
waitReadyToFinish(commit).whenComplete((ignored, readyException) -> {
try {
if (commit) {
if (readyException == null) {
CompletableFuture<Void> actionFut =
finishAction.apply(true);
actionFut.whenComplete((ignoredFinishActionResult,
finishException) ->
completeFinishInProgressFuture(true, null,
finishException));
} else {
// If we got ready exception, that means some of enlisted
partitions could be broken/unavailable.
// Respond to caller with the commit failure immediately to
reduce potential unavailability window.
completeFinishInProgressFuture(true, readyException, null);
finishAction.apply(false); <<< !!! tx clean-up is triggered
after we completed finish future
}
return;
}
{code}
was:
{noformat}
java.lang.AssertionError: Metric name = PendingWriteIntentsExpected:
is <0L> but: was <1L>
Expected :is <0L>
Actual :<1L>
at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20)
at
org.apache.ignite.internal.tx.ItTransactionMetricsTest.testMetricValues(ItTransactionMetricsTest.java:123)
at
org.apache.ignite.internal.tx.ItTransactionMetricsTest.testRollbackTransactionOnLeaseExpiration(ItTransactionMetricsTest.java:368)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
at java.base/java.util.ArrayList.forEach(ArrayList.java:1541) {noformat}
It seems to me, that the root cause is that the client future
> ItTransactionMetricsTest.testRollbackTransactionOnLeaseExpiration is flaky
> --------------------------------------------------------------------------
>
> Key: IGNITE-27812
> URL: https://issues.apache.org/jira/browse/IGNITE-27812
> Project: Ignite
> Issue Type: Bug
> Components: transactions ai3
> Reporter: Vyacheslav Koptilin
> Assignee: Vyacheslav Koptilin
> Priority: Major
> Labels: MakeTeamcityGreenAgain, ignite-3
> Time Spent: 10m
> Remaining Estimate: 0h
>
> {noformat}
> java.lang.AssertionError: Metric name = PendingWriteIntentsExpected:
> is <0L> but: was <1L>
> Expected :is <0L>
> Actual :<1L>
> at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20)
> at
> org.apache.ignite.internal.tx.ItTransactionMetricsTest.testMetricValues(ItTransactionMetricsTest.java:123)
>
> at
> org.apache.ignite.internal.tx.ItTransactionMetricsTest.testRollbackTransactionOnLeaseExpiration(ItTransactionMetricsTest.java:368)
>
> at java.base/java.lang.reflect.Method.invoke(Method.java:566)
> at java.base/java.util.ArrayList.forEach(ArrayList.java:1541)
> at java.base/java.util.ArrayList.forEach(ArrayList.java:1541) {noformat}
>
> It seems to me, that the root cause is that the client's future might be
> completed before we initiate the transaction clean-up process that should
> release locks and switch write intents. See
> `TransactionInflights.preformFinish()`:
> {code:title=TransactionInflights.preformFinish()|language=java}
> CompletableFuture<Void> performFinish(boolean commit, Function<Boolean,
> CompletableFuture<Void>> finishAction) {
> waitReadyToFinish(commit).whenComplete((ignored, readyException) -> {
> try {
> if (commit) {
> if (readyException == null) {
> CompletableFuture<Void> actionFut =
> finishAction.apply(true);
> actionFut.whenComplete((ignoredFinishActionResult,
> finishException) ->
> completeFinishInProgressFuture(true, null,
> finishException));
> } else {
> // If we got ready exception, that means some of enlisted
> partitions could be broken/unavailable.
> // Respond to caller with the commit failure immediately
> to reduce potential unavailability window.
> completeFinishInProgressFuture(true, readyException,
> null);
> finishAction.apply(false); <<< !!! tx clean-up is
> triggered after we completed finish future
> }
> return;
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)