You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by gv...@apache.org on 2019/01/22 11:04:59 UTC

[ignite] branch master updated: IGNITE-10763: MVCC: Fixed transaction already completed error in certain scenarios. This closes #5770.

This is an automated email from the ASF dual-hosted git repository.

gvvinblade pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 23b4b29  IGNITE-10763: MVCC: Fixed transaction already completed error in certain scenarios. This closes #5770.
23b4b29 is described below

commit 23b4b29b2dcec97b3b080f40094a0b771bcbbe7b
Author: ipavlukhin <vo...@gmail.com>
AuthorDate: Tue Jan 22 14:04:27 2019 +0300

    IGNITE-10763: MVCC: Fixed transaction already completed error in certain scenarios. This closes #5770.
---
 .../near/GridNearTxAbstractEnlistFuture.java       |  2 +-
 .../cache/distributed/near/GridNearTxLocal.java    | 45 ++++++++++++----------
 .../near/GridNearTxSelectForUpdateFuture.java      |  2 +-
 .../query/h2/DmlStatementsProcessor.java           |  3 +-
 .../processors/query/h2/IgniteH2Indexing.java      |  5 ++-
 .../query/h2/twostep/GridReduceQueryExecutor.java  |  2 +-
 .../mvcc/CacheMvccReplicatedSqlTxQueriesTest.java  |  2 -
 ...cheMvccSqlTxQueriesWithReducerAbstractTest.java |  2 -
 8 files changed, 33 insertions(+), 30 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
index afed6e0..b31314d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
@@ -382,7 +382,7 @@ public abstract class GridNearTxAbstractEnlistFuture<T> extends GridCacheCompoun
             return false;
 
         // Need to unlock topology to avoid deadlock with binary descriptors registration.
-        if(cctx.topology().holdsLock())
+        if (cctx.topology().holdsLock())
             cctx.topology().readUnlock();
 
         cctx.tm().txContext(tx);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 08dfe2a..134412c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -2086,32 +2086,37 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
      * @param fut Enlist future.
      * @return Operation future.
      */
-    public IgniteInternalFuture<Long> updateAsync(GridNearTxQueryAbstractEnlistFuture fut) {
-        fut.init();
+    private IgniteInternalFuture<Long> updateAsync(GridNearTxQueryAbstractEnlistFuture fut) {
+        try {
+            fut.init();
 
-        return nonInterruptable(new GridEmbeddedFuture<>(fut.chain(new CX1<IgniteInternalFuture<Long>, Boolean>() {
-            @Override public Boolean applyx(IgniteInternalFuture<Long> fut0) throws IgniteCheckedException {
-                return fut0.get() != null;
-            }
-        }), new PLC1<Long>(null) {
-            @Override protected Long postLock(Long val) throws IgniteCheckedException {
-                Long res = fut.get();
+            return nonInterruptable(new GridEmbeddedFuture<>(fut.chain(new CX1<IgniteInternalFuture<Long>, Boolean>() {
+                @Override public Boolean applyx(IgniteInternalFuture<Long> fut0) throws IgniteCheckedException {
+                    return fut0.get() != null;
+                }
+            }), new PLC1<Long>(null) {
+                @Override protected Long postLock(Long val) throws IgniteCheckedException {
+                    Long res = fut.get();
 
-                assert mvccSnapshot != null;
-                assert res != null;
+                    assert mvccSnapshot != null;
+                    assert res != null;
 
-                if (res > 0) {
-                    if (mvccSnapshot.operationCounter() == ~MvccUtils.MVCC_OP_COUNTER_MASK) {
-                        throw new IgniteCheckedException("The maximum limit of the number of statements allowed in" +
-                            " one transaction is reached. [max=" + mvccSnapshot.operationCounter() + ']');
+                    if (res > 0) {
+                        if (mvccSnapshot.operationCounter() == ~MvccUtils.MVCC_OP_COUNTER_MASK) {
+                            throw new IgniteCheckedException("The maximum limit of the number of statements allowed in" +
+                                " one transaction is reached. [max=" + mvccSnapshot.operationCounter() + ']');
+                        }
+
+                        mvccSnapshot.incrementOperationCounter();
                     }
 
-                    mvccSnapshot.incrementOperationCounter();
+                    return res;
                 }
-
-                return res;
-            }
-        }));
+            }));
+        }
+        finally {
+            cctx.tm().resetContext();
+        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java
index a5ab7cd..e513a44 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxSelectForUpdateFuture.java
@@ -180,7 +180,7 @@ public class GridNearTxSelectForUpdateFuture extends GridCacheCompoundIdentityFu
         if (!DONE_UPD.compareAndSet(this, 0, 1))
             return false;
 
-        cctx.tm().txContext(tx);
+        cctx.tm().resetContext();
 
         Throwable ex0 = ex;
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 5873bd7..a51a058 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -99,7 +99,6 @@ import org.h2.command.dml.Merge;
 import org.h2.command.dml.Update;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkActive;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccTracker;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.requestSnapshot;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx;
@@ -537,7 +536,7 @@ public class DmlStatementsProcessor {
             if (implicit)
                 tx = txStart(cctx, fieldsQry.getTimeout());
 
-            requestSnapshot(cctx, checkActive(tx));
+            requestSnapshot(cctx, tx);
 
             try (GridNearTxLocal toCommit = commit ? tx : null) {
                 long timeout = implicit
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index a926c23..abc24ad 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1192,8 +1192,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
             GridNearTxLocal tx = tracker != null ? tx(ctx) : null;
 
-            if (qry.forUpdate())
+            if (qry.forUpdate()) {
+                // Locking has no meaning if SELECT FOR UPDATE is not executed in explicit transaction.
+                // So, we can can reset forUpdate flag if there is no explicit transaction.
                 qry.forUpdate(checkActive(tx) != null);
+            }
 
             int opTimeout = operationTimeout(qryTimeout, tx);
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 41d1f61..fd5eee2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -107,8 +107,8 @@ import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.checkAc
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccEnabled;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.tx;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery.EMPTY_PARAMS;
-import static org.apache.ignite.internal.processors.query.h2.opt.join.DistributedJoinMode.OFF;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE;
+import static org.apache.ignite.internal.processors.query.h2.opt.join.DistributedJoinMode.OFF;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter.mergeTableIdentifier;
 import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest.setDataPageScanEnabled;
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java
index 4554a7f..7dce522 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccReplicatedSqlTxQueriesTest.java
@@ -28,7 +28,6 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.transactions.Transaction;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -151,7 +150,6 @@ public class CacheMvccReplicatedSqlTxQueriesTest extends CacheMvccSqlTxQueriesAb
      *
      * @throws Exception If failed.
      */
-    @Ignore("https://issues.apache.org/jira/browse/IGNITE-10763")
     @Test
     public void testReplicatedAndPartitionedUpdateSingleTransaction() throws Exception {
         ccfgs = new CacheConfiguration[] {
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java
index f0a586a..d4ad094 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlTxQueriesWithReducerAbstractTest.java
@@ -39,7 +39,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -535,7 +534,6 @@ public abstract class CacheMvccSqlTxQueriesWithReducerAbstractTest extends Cache
     /**
      * @throws Exception If failed.
      */
-    @Ignore("https://issues.apache.org/jira/browse/IGNITE-10763")
     @Test
     public void testQueryReducerDeadlockInsertWithTxTimeout() throws Exception {
         checkQueryReducerDeadlockInsert(TimeoutMode.TX);