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/11/13 12:01:45 UTC

ignite git commit: IGNITE-6044: SQL: Disallow DML inside active non-MVCC transactions. This is needed to avoid deadlocks as updated order is not known. Users may skip this check with ""-DIGNITE_ALLOW_DML_INSIDE_TRANSACTION=true". This closes #4613.

Repository: ignite
Updated Branches:
  refs/heads/master ded41e9d1 -> 0f387681b


IGNITE-6044: SQL: Disallow DML inside active non-MVCC transactions. This is needed to avoid deadlocks as updated order is not known. Users may skip this check with ""-DIGNITE_ALLOW_DML_INSIDE_TRANSACTION=true". This closes #4613.


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

Branch: refs/heads/master
Commit: 0f387681b81d41ba2f56f186eca3308ccd11d4b9
Parents: ded41e9
Author: Yuriy Gerzhedovich <yg...@gridgain.com>
Authored: Tue Nov 13 15:01:37 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Nov 13 15:01:37 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   8 +
 .../ignite/testframework/GridTestUtils.java     |  38 +++
 .../query/h2/DmlStatementsProcessor.java        |  13 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |  30 ++-
 .../transaction/DmlInsideTransactionTest.java   | 239 +++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java        |   2 +
 6 files changed, 322 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index ef416c0..3f71642 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -983,6 +983,14 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_EVICTION_PERMITS = "IGNITE_EVICTION_PERMITS";
 
     /**
+     * When set to {@code true}, Ignite will allow execute DML operation (MERGE|INSERT|UPDATE|DELETE)
+     * within transaction for non MVCC mode.
+     *
+     * Default is {@code false}.
+     */
+    public static final String IGNITE_ALLOW_DML_INSIDE_TRANSACTION = "IGNITE_ALLOW_DML_INSIDE_TRANSACTION";
+
+    /**
      * Timeout between ZooKeeper client retries, default 2s.
      */
     public static final String IGNITE_ZOOKEEPER_DISCOVERY_RETRY_TIMEOUT = "IGNITE_ZOOKEEPER_DISCOVERY_RETRY_TIMEOUT";

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index e82a0b0..dad5344 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -43,6 +43,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Properties;
 import java.util.Queue;
 import java.util.Random;
 import java.util.Set;
@@ -2041,4 +2042,41 @@ public final class GridTestUtils {
             return Math.min((int) (TEST_SCALE_FACTOR_VALUE * val), upperBound);
         }
     }
+
+    /** Adds system property on initialization and removes it when closed. */
+    public static final class SystemProperty implements AutoCloseable {
+        /** Name of property. */
+        private final String name;
+
+        /** Original value of property. */
+        private final String originalValue;
+
+        /**
+         * Constructor.
+         *
+         * @param name Name.
+         * @param val Value.
+         */
+        public SystemProperty(String name, String val) {
+            this.name = name;
+
+            Properties props = System.getProperties();
+
+            originalValue = (String)props.put(name, val);
+
+            System.setProperties(props);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            Properties props = System.getProperties();
+
+            if (originalValue != null)
+                props.put(name, originalValue);
+            else
+                props.remove(name);
+
+            System.setProperties(props);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
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 363b173..9b31b02 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
@@ -39,6 +39,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.query.BulkLoadContextCursor;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -120,6 +121,16 @@ public class DmlStatementsProcessor {
     /** Default size for update plan cache. */
     private static final int PLAN_CACHE_SIZE = 1024;
 
+    /** Cached value of {@code IgniteSystemProperties.IGNITE_ALLOW_DML_INSIDE_TRANSACTION}. */
+    private final boolean isDmlAllowedOverride;
+
+    /**
+     * Default constructor.
+     */
+    public DmlStatementsProcessor() {
+        isDmlAllowedOverride = Boolean.getBoolean(IgniteSystemProperties.IGNITE_ALLOW_DML_INSIDE_TRANSACTION);
+    }
+
     /** Update plans cache. */
     private final ConcurrentMap<H2CachedStatementKey, UpdatePlan> planCache =
         new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
@@ -737,7 +748,7 @@ public class DmlStatementsProcessor {
         if (res != null)
             return res;
 
-        res = UpdatePlanBuilder.planForStatement(p, loc, idx, conn, fieldsQry, errKeysPos);
+        res = UpdatePlanBuilder.planForStatement(p, loc, idx, conn, fieldsQry, errKeysPos, isDmlAllowedOverride);
 
         // Don't cache re-runs
         if (errKeysPos == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index d4be92c..477f22a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -94,7 +94,8 @@ public final class UpdatePlanBuilder {
      * @return Update plan.
      */
     public static UpdatePlan planForStatement(Prepared prepared, boolean loc, IgniteH2Indexing idx,
-        @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQry, @Nullable Integer errKeysPos)
+        @Nullable Connection conn, @Nullable SqlFieldsQuery fieldsQry, @Nullable Integer errKeysPos,
+        boolean dmlInsideTxAllowed)
         throws IgniteCheckedException {
         assert !prepared.isQuery();
 
@@ -104,14 +105,15 @@ public final class UpdatePlanBuilder {
 
         boolean mvccEnabled = false;
 
-        GridCacheContext cctx = null;
+        GridCacheContext prevCctx = null;
 
-        // check all involved caches
         for (Object o : parser.objectsMap().values()) {
             if (o instanceof GridSqlInsert)
                 o = ((GridSqlInsert)o).into();
             else if (o instanceof GridSqlMerge)
                 o = ((GridSqlMerge)o).into();
+            else if (o instanceof GridSqlUpdate)
+                o = ((GridSqlUpdate)o).target();
             else if (o instanceof GridSqlDelete)
                 o = ((GridSqlDelete)o).from();
 
@@ -124,10 +126,24 @@ public final class UpdatePlanBuilder {
                         ((GridSqlTable)o).tableName() + "'", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
                 }
 
-                if (cctx == null)
-                    mvccEnabled = (cctx = (((GridSqlTable)o).dataTable()).cache()).mvccEnabled();
-                else if (((GridSqlTable)o).dataTable().cache().mvccEnabled() != mvccEnabled)
-                    MvccUtils.throwAtomicityModesMismatchException(cctx, ((GridSqlTable)o).dataTable().cache());
+                if (prevCctx == null) {
+                    prevCctx = (((GridSqlTable)o).dataTable()).cache();
+
+                    mvccEnabled = prevCctx.mvccEnabled();
+
+                    if (!mvccEnabled && !dmlInsideTxAllowed && prevCctx.cache().context().tm().inUserTx()) {
+                        throw new IgniteSQLException("DML statements are not allowed inside a transaction over " +
+                            "cache(s) with TRANSACTIONAL atomicity mode (change atomicity mode to " +
+                            "TRANSACTIONAL_SNAPSHOT or disable this error message with system property " +
+                            "\"IGNITE_ALLOW_DML_INSIDE_TRANSACTION\" [cacheName=" + prevCctx.name() + ']');
+                    }
+                }
+                else {
+                    GridCacheContext cctx = ((GridSqlTable)o).dataTable().cache();
+
+                    if (cctx.mvccEnabled() != mvccEnabled)
+                        MvccUtils.throwAtomicityModesMismatchException(prevCctx, cctx);
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/transaction/DmlInsideTransactionTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/transaction/DmlInsideTransactionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/transaction/DmlInsideTransactionTest.java
new file mode 100644
index 0000000..633bb6f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/transaction/DmlInsideTransactionTest.java
@@ -0,0 +1,239 @@
+/*
+ * 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.transaction;
+
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.query.Query;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils.SystemProperty;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+
+/**
+ * Tests DML allow/disallow operation inside transaction.
+ */
+public class DmlInsideTransactionTest extends GridCommonAbstractTest {
+    /** Person cache name. */
+    private static final String CACHE_PERSON = "PersonCache";
+
+    /** Set of DML queries for tests. */
+    private static final String[] DML_QUERIES = {
+        "MERGE INTO TEST.Person(id, name, orgId) VALUES(111,'NAME',111)",
+        "INSERT INTO TEST.Person(id, name, orgId) VALUES(222,'NAME',111)",
+        "UPDATE TEST.Person SET name='new name'",
+        "DELETE TEST.Person WHERE id=1",
+        "INSERT INTO TEST.Person(id, name, orgId) SELECT id+1000, name, orgId FROM TEST.Person"
+    };
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Checking correct behaviour for DML inside transaction by default.
+     *
+     * @throws Exception In case failure.
+     */
+    public void testDmlInTransactionByDefault() throws Exception {
+        prepareIgnite();
+
+        for (String dmlQuery : DML_QUERIES) {
+            runDmlSqlFieldsQueryInTransactionTest(dmlQuery, false, false);
+
+            runDmlSqlFieldsQueryInTransactionTest(dmlQuery, true, false);
+        }
+    }
+
+    /**
+     * Checking correct behaviour for DML inside transaction when compatibility property set as disabled.
+     *
+     * @throws Exception In case failure.
+     */
+    public void testDmlInTransactionInDisabledCompatibilityMode() throws Exception {
+        try (SystemProperty ignored = new SystemProperty(IgniteSystemProperties.IGNITE_ALLOW_DML_INSIDE_TRANSACTION, "false")) {
+            prepareIgnite();
+
+            for (String dmlQuery : DML_QUERIES) {
+                runDmlSqlFieldsQueryInTransactionTest(dmlQuery, false, false);
+
+                runDmlSqlFieldsQueryInTransactionTest(dmlQuery, true, false);
+            }
+        }
+    }
+
+    /**
+     * Checking correct behaviour for DML inside transaction when compatibility property set as enabled.
+     *
+     * @throws Exception In case failure.
+     */
+    public void testDmlInTransactionInCompatibilityMode() throws Exception {
+        try (SystemProperty ignored = new SystemProperty(IgniteSystemProperties.IGNITE_ALLOW_DML_INSIDE_TRANSACTION, "true")) {
+            prepareIgnite();
+
+            for (String dmlQuery : DML_QUERIES) {
+                runDmlSqlFieldsQueryInTransactionTest(dmlQuery, false, true);
+
+                runDmlSqlFieldsQueryInTransactionTest(dmlQuery, true, true);
+            }
+        }
+    }
+
+    /**
+     * Checking that DML can be executed without a errors outside transaction.
+     *
+     * @throws Exception In case failure.
+     */
+    public void testDmlNotInTransaction() throws Exception {
+        prepareIgnite();
+
+        for (String dmlQuery : DML_QUERIES) {
+            grid(0).cache(CACHE_PERSON).query(new SqlFieldsQuery(dmlQuery));
+
+            grid(0).cache(CACHE_PERSON).clear();
+
+            grid(0).cache(CACHE_PERSON).query(new SqlFieldsQuery(dmlQuery).setLocal(true));
+        }
+    }
+
+    /**
+     * Start Ignite grid and create cache.
+     *
+     * @throws Exception In case is failure.
+     */
+    private void prepareIgnite() throws Exception {
+        IgniteEx ignite = startGrid(0);
+
+        ignite.createCache(new CacheConfiguration<PersonKey, Person>()
+            .setName(CACHE_PERSON)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setSqlSchema("TEST")
+            .setIndexedTypes(PersonKey.class, Person.class));
+    }
+
+    /**
+     * Run DML query as SqlFieldsQuery and check that DML is not allowed or not inside transaction. Also checked that
+     * using DML will not lead to rollback
+     *
+     * @param dmlQry Dml query which should be executed in transaction.
+     * @param isLocal Is local query.
+     * @param isAllowed true in case DML should work inside transaction, false otherwise.
+     */
+    private void runDmlSqlFieldsQueryInTransactionTest(String dmlQry, boolean isLocal, boolean isAllowed) {
+        SqlFieldsQuery query = new SqlFieldsQuery(dmlQry).setLocal(isLocal);
+        runDmlInTransactionTest(query, isAllowed);
+    }
+
+    /**
+     * Run DML query and check that DML is not allowed or not inside transaction. Also checked that using DML will not
+     * lead to rollback.
+     *
+     * @param query Query with DML operation to be run.
+     * @param isAllowed true in case DML should work inside transaction, false otherwise.
+     */
+    private void runDmlInTransactionTest(Query query, boolean isAllowed) {
+        IgniteEx ignite = grid(0);
+
+        IgniteCache<PersonKey, Person> cache = ignite.cache(CACHE_PERSON);
+
+        cache.removeAll();
+
+        assertEquals(0, cache.query(new SqlFieldsQuery("SELECT * FROM TEST.Person")).getAll().size());
+
+        try (Transaction tx = ignite.transactions().txStart()) {
+            cache.put(new PersonKey(1L), new Person("person", 2));
+
+            if (isAllowed)
+                cache.query(query);
+            else {
+                assertThrows(log, () -> {
+                    cache.query(query);
+
+                    return null;
+                }, CacheException.class, "DML statements are not allowed inside a transaction over cache(s) with TRANSACTIONAL atomicity");
+            }
+
+            tx.commit();
+        }
+
+        assertTrue(!cache.query(new SqlFieldsQuery("SELECT * FROM TEST.Person")).getAll().isEmpty());
+    }
+
+    /**
+     * Person key.
+     */
+    public static class PersonKey {
+        /** ID. */
+        @QuerySqlField
+        public long id;
+
+        /**
+         * Constructor.
+         *
+         * @param id ID.
+         */
+        PersonKey(long id) {
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return (int)id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            return obj instanceof PersonKey && (F.eq(id, ((PersonKey)obj).id));
+        }
+    }
+
+    /**
+     * Person.
+     */
+    public static class Person {
+        /** Name. */
+        @QuerySqlField
+        public String name;
+
+        /** Organization ID. */
+        @QuerySqlField(index = true)
+        public long orgId;
+
+        /**
+         * Constructor.
+         *
+         * @param name Name.
+         * @param orgId Organization ID.
+         */
+        public Person(String name, long orgId) {
+            this.name = name;
+            this.orgId = orgId;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0f387681/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index f1ff5e6..8213380 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -168,6 +168,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDe
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryWithH2IndexingSelfTest;
+import org.apache.ignite.internal.processors.cache.transaction.DmlInsideTransactionTest;
 import org.apache.ignite.internal.processors.client.ClientConnectorConfigurationValidationSelfTest;
 import org.apache.ignite.internal.processors.database.baseline.IgniteStableBaselineBinObjFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.query.IgniteCachelessQueriesSelfTest;
@@ -240,6 +241,7 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
         IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite");
 
         suite.addTestSuite(AffinityKeyNameAndValueFieldNameConflictTest.class);
+        suite.addTestSuite(DmlInsideTransactionTest.class);
 
         suite.addTestSuite(PartitionedSqlTest.class);
         suite.addTestSuite(ReplicatedSqlTest.class);