You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/10/11 07:42:19 UTC

ignite git commit: IGNITE-9133: MVCC: do not add transaction without enlisted key to completed versions set.

Repository: ignite
Updated Branches:
  refs/heads/master 824939ba1 -> ccf9c1ce3


IGNITE-9133: MVCC: do not add transaction without enlisted key to completed versions set.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/ccf9c1ce
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ccf9c1ce
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ccf9c1ce

Branch: refs/heads/master
Commit: ccf9c1ce33b5513372beed0ee8463ed3684e35e9
Parents: 824939b
Author: devozerov <vo...@gridgain.com>
Authored: Thu Oct 11 10:42:10 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 11 10:42:10 2018 +0300

----------------------------------------------------------------------
 .../dht/NearTxQueryEnlistResultHandler.java     |   4 +-
 .../transactions/IgniteTxLocalAdapter.java      |  13 ++-
 .../index/MvccEmptyTransactionSelfTest.java     | 110 +++++++++++++++++++
 .../testsuites/IgniteCacheMvccSqlTestSuite.java |   2 +
 4 files changed, 126 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ccf9c1ce/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java
index 8043def..2058377 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/NearTxQueryEnlistResultHandler.java
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQu
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxQueryResultsEnlistResponse;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -126,7 +125,8 @@ public final class NearTxQueryEnlistResultHandler implements CI1<IgniteInternalF
         GridNearTxQueryEnlistResponse res = createResponse(fut);
 
         if (res.removeMapping()) {
-            // TODO IGNITE-9133
+            tx.forceSkipCompletedVersions();
+
             tx.rollbackDhtLocalAsync().listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
                 @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut0) {
                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ccf9c1ce/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index d0e3dca..b7624db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -167,6 +167,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     /** */
     private volatile boolean qryEnlisted;
 
+    /** Whether to skip update of completed versions map during rollback caused by empty update set in MVCC TX. */
+    private boolean forceSkipCompletedVers;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -1144,7 +1147,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         }
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
-            cctx.tm().rollbackTx(this, clearThreadMap, false);
+            cctx.tm().rollbackTx(this, clearThreadMap, forceSkipCompletedVers);
 
             if (!internal()) {
                 Collection<CacheStoreManager> stores = txState.stores(cctx);
@@ -1163,6 +1166,14 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     }
 
     /**
+     * Forces transaction to skip update of completed versions map during rollback caused by empty update set
+     * in MVCC TX.
+     */
+    public void forceSkipCompletedVersions() {
+        forceSkipCompletedVers = true;
+    }
+
+    /**
      * @param ctx Cache context.
      * @param key Key.
      * @param expiryPlc Expiry policy.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ccf9c1ce/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/MvccEmptyTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/MvccEmptyTransactionSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/MvccEmptyTransactionSelfTest.java
new file mode 100644
index 0000000..5a66062
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/MvccEmptyTransactionSelfTest.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import java.util.List;
+
+/**
+ * Test for empty transaction while is then enlisted with real value.
+ */
+public class MvccEmptyTransactionSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testEmptyTransaction() throws Exception {
+        Ignition.start(config("srv", false));
+
+        Ignite cli = Ignition.start(config("cli", true));
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1:10801")) {
+            try (Statement stmt = conn.createStatement()) {
+                stmt.execute("CREATE TABLE person (id BIGINT PRIMARY KEY, name VARCHAR) " +
+                    "WITH \"atomicity=TRANSACTIONAL_SNAPSHOT, cache_name=PERSON_CACHE\"");
+            }
+        }
+
+        IgniteCache cache = cli.cache("PERSON_CACHE");
+
+        try (Transaction tx = cli.transactions().txStart()) {
+            // This will cause empty near TX to be created and then rolled back.
+            cache.query(new SqlFieldsQuery("UPDATE person SET name=?").setArgs("Petr")).getAll();
+
+            // One more time.
+            cache.query(new SqlFieldsQuery("UPDATE person SET name=?").setArgs("Petr")).getAll();
+
+            // Normal transaction is created, and several updates are performed.
+            cache.query(new SqlFieldsQuery("INSERT INTO person VALUES (?, ?)").setArgs(1, "Ivan")).getAll();
+            cache.query(new SqlFieldsQuery("UPDATE person SET name=?").setArgs("Sergey")).getAll();
+
+            // Another update with empty response.
+            cache.query(new SqlFieldsQuery("UPDATE person SET name=? WHERE name=?").setArgs("Vasiliy", "Ivan")).getAll();
+
+            // One more normal update.
+            cache.query(new SqlFieldsQuery("UPDATE person SET name=?").setArgs("Vsevolod")).getAll();
+
+            tx.commit();
+        }
+
+        List<List<Object>> res = cache.query(new SqlFieldsQuery("SELECT name FROM person")).getAll();
+
+        assert res.size() == 1;
+        assert res.get(0).size() == 1;
+
+        assertEquals("Vsevolod", (String)res.get(0).get(0));
+    }
+
+    /**
+     * Create config.
+     *
+     * @param name Name.
+     * @param client Client flag.
+     * @return Config.
+     */
+    private static IgniteConfiguration config(String name, boolean client) {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setIgniteInstanceName(name);
+        cfg.setClientMode(client);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ccf9c1ce/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
index d8ea226..b5cb3e0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.index.MvccEmptyTransactionSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SqlTransactionsCommandsWithMvccEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBulkLoadTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccDmlSimpleTest;
@@ -58,6 +59,7 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("IgniteCache SQL MVCC Test Suite");
 
         // Simple tests.
+        suite.addTestSuite(MvccEmptyTransactionSelfTest.class);
         suite.addTestSuite(CacheMvccSqlConfigurationValidationTest.class);
         suite.addTestSuite(CacheMvccDmlSimpleTest.class);
         suite.addTestSuite(SqlTransactionsCommandsWithMvccEnabledSelfTest.class);