You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/11 12:26:29 UTC

[1/4] incubator-ignite git commit: #sberb-28: add test for starting cache in transaction.

Repository: incubator-ignite
Updated Branches:
  refs/heads/sberb-28 [created] 9e949c83b


#sberb-28: add test for starting cache in transaction.


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

Branch: refs/heads/sberb-28
Commit: d75758256dcc2579c915a24d1fd4a8d89b7c8fe5
Parents: 89a4f7c
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 12:41:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 12:41:51 2015 +0300

----------------------------------------------------------------------
 .../cache/StartCacheInTransactionSelfTest.java  | 74 ++++++++++++++++++++
 1 file changed, 74 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7575825/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
new file mode 100644
index 0000000..391a2c6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+/**
+ * Check starting cache in transaction.
+ */
+public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartCache() throws Exception {
+        Ignite ignite = grid(0);
+
+        final String key = "key";
+        final String val = "val";
+
+        try (Transaction tx = ignite.transactions().txStart(
+            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
+            ignite.cache(null).put(key, val);
+
+            IgniteCache<String, String> cache = ignite.createCache("NEW_CACHE");
+
+            cache.put(key, val);
+
+            tx.commit();
+        }
+    }
+}


[4/4] incubator-ignite git commit: #sberb-28: add test to test suite.

Posted by sb...@apache.org.
#sberb-28: add test to test suite.


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

Branch: refs/heads/sberb-28
Commit: 9e949c83bea1218596a8e864c41d3b4873b55bf5
Parents: beba207
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 13:26:12 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 13:26:12 2015 +0300

----------------------------------------------------------------------
 .../IgniteStartCacheInTransactionSelfTest.java  | 130 +++++++++++++++++++
 .../cache/StartCacheInTransactionSelfTest.java  | 129 ------------------
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 3 files changed, 132 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e949c83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
new file mode 100644
index 0000000..815b4b9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Check starting cache in transaction.
+ */
+public class IgniteStartCacheInTransactionSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartCache() throws Exception {
+        final Ignite ignite = grid(0);
+
+        final String key = "key";
+        final String val = "val";
+
+        try (Transaction tx = ignite.transactions().txStart(
+            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
+            ignite.cache(null).put(key, val);
+
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    ignite.createCache("NEW_CACHE");
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopCache() throws Exception {
+        final Ignite ignite = grid(0);
+
+        final String key = "key";
+        final String val = "val";
+
+        try (Transaction tx = ignite.transactions().txStart(
+            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
+            ignite.cache(null).put(key, val);
+
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    ignite.destroyCache(null);
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockCache() throws Exception {
+        final Ignite ignite = grid(0);
+
+        final String key = "key";
+
+        Lock lock = ignite.cache(null).lock(key);
+
+        lock.lock();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite.createCache("NEW_CACHE");
+
+                return null;
+            }
+        }, IgniteException.class, "Cannot start/stop cache within lock.");
+
+        lock.unlock();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e949c83/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
deleted file mode 100644
index 7518a62..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.apache.ignite.transactions.*;
-
-import java.util.concurrent.*;
-import java.util.concurrent.locks.*;
-
-/**
- * Check starting cache in transaction.
- */
-public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg =  super.getConfiguration(gridName);
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setBackups(1);
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStartCache() throws Exception {
-        final Ignite ignite = grid(0);
-
-        final String key = "key";
-        final String val = "val";
-
-        try (Transaction tx = ignite.transactions().txStart(
-            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
-            ignite.cache(null).put(key, val);
-
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    ignite.createCache("NEW_CACHE");
-
-                    return null;
-                }
-            }, IgniteException.class, "Cannot start/stop cache within transaction.");
-
-            tx.commit();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStopCache() throws Exception {
-        final Ignite ignite = grid(0);
-
-        final String key = "key";
-        final String val = "val";
-
-        try (Transaction tx = ignite.transactions().txStart(
-            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
-            ignite.cache(null).put(key, val);
-
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    ignite.destroyCache(null);
-
-                    return null;
-                }
-            }, IgniteException.class, "Cannot start/stop cache within transaction.");
-
-            tx.commit();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLockCache() throws Exception {
-        final Ignite ignite = grid(0);
-
-        final String key = "key";
-
-        Lock lock = ignite.cache(null).lock(key);
-
-        lock.lock();
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                ignite.createCache("NEW_CACHE");
-
-                return null;
-            }
-        }, IgniteException.class, "Cannot start/stop cache within lock.");
-
-        lock.unlock();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e949c83/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index ed9fc9a..2cbb00d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -140,6 +140,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(IgniteCacheManyClientsTest.class);
 
+        suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
+
         return suite;
     }
 }


[2/4] incubator-ignite git commit: #sberb-28: add exception in start/stop cache if transaction exist.

Posted by sb...@apache.org.
#sberb-28: add exception in start/stop cache if transaction exist.


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

Branch: refs/heads/sberb-28
Commit: 4f18b6c9e1be20ee2629aef190e449c78327f802
Parents: d757582
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 13:00:23 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 13:00:23 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 14 +++++++
 .../cache/StartCacheInTransactionSelfTest.java  | 40 ++++++++++++++++++--
 2 files changed, 51 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f18b6c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 5582ba7..17e6ec0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1880,6 +1880,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         CacheType cacheType,
         boolean failIfExists
     ) {
+        checkEmptyTransactions();
+
         assert ccfg != null || nearCfg != null;
 
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
@@ -1970,6 +1972,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is stopped.
      */
     public IgniteInternalFuture<?> dynamicStopCache(String cacheName) {
+        checkEmptyTransactions();
+
         DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId(), true);
 
         return F.first(initiateCacheChanges(F.asList(t), false));
@@ -2653,6 +2657,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private IgniteCacheProxy startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
+        checkEmptyTransactions();
+
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2950,6 +2956,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @throws IgniteException If transaction exist.
+     */
+    private void checkEmptyTransactions() throws IgniteException {
+        if (transactions().tx() != null)
+            throw new IgniteException("Cannot start/stop cache within transaction.");
+    }
+
+    /**
      * @param val Object to check.
      * @throws IgniteCheckedException If validation failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f18b6c9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
index 391a2c6..e2be184 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
@@ -20,9 +20,12 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
 
+import java.util.concurrent.*;
+
 /**
  * Check starting cache in transaction.
  */
@@ -55,7 +58,34 @@ public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartCache() throws Exception {
-        Ignite ignite = grid(0);
+        final Ignite ignite = grid(0);
+
+        final String key = "key";
+        final String val = "val";
+
+        try (Transaction tx = ignite.transactions().txStart(
+            TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
+            ignite.cache(null).put(key, val);
+
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    IgniteCache<String, String> cache = ignite.createCache("NEW_CACHE");
+
+                    cache.put(key, val);
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopCache() throws Exception {
+        final Ignite ignite = grid(0);
 
         final String key = "key";
         final String val = "val";
@@ -64,9 +94,13 @@ public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
             TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)){
             ignite.cache(null).put(key, val);
 
-            IgniteCache<String, String> cache = ignite.createCache("NEW_CACHE");
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    ignite.destroyCache(null);
 
-            cache.put(key, val);
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
 
             tx.commit();
         }


[3/4] incubator-ignite git commit: #sberb-28: add exception in start/stop cache if lock exist.

Posted by sb...@apache.org.
#sberb-28: add exception in start/stop cache if lock exist.


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

Branch: refs/heads/sberb-28
Commit: beba207454ac124f02945e7447a6993cce56e951
Parents: 4f18b6c
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 13:24:42 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 13:24:42 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMvccManager.java  |  7 +++++
 .../processors/cache/GridCacheProcessor.java    |  3 +++
 .../cache/StartCacheInTransactionSelfTest.java  | 27 +++++++++++++++++---
 3 files changed, 34 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/beba2074/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index c528e08..bd48f5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -572,6 +572,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @return True if any locks acquired.
+     */
+    public boolean hasLocks() {
+        return !locked.isEmpty() || !nearLocked.isEmpty();
+    }
+
+    /**
      * This method has poor performance, so use with care. It is currently only used by {@code DGC}.
      *
      * @return Remote candidates.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/beba2074/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 17e6ec0..6c320ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2961,6 +2961,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private void checkEmptyTransactions() throws IgniteException {
         if (transactions().tx() != null)
             throw new IgniteException("Cannot start/stop cache within transaction.");
+
+        if (ctx.cache().context().mvcc().hasLocks())
+            throw new IgniteException("Cannot start/stop cache within lock.");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/beba2074/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
index e2be184..7518a62 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/StartCacheInTransactionSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
 
 import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
 
 /**
  * Check starting cache in transaction.
@@ -69,9 +70,7 @@ public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
 
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    IgniteCache<String, String> cache = ignite.createCache("NEW_CACHE");
-
-                    cache.put(key, val);
+                    ignite.createCache("NEW_CACHE");
 
                     return null;
                 }
@@ -105,4 +104,26 @@ public class StartCacheInTransactionSelfTest extends GridCommonAbstractTest {
             tx.commit();
         }
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockCache() throws Exception {
+        final Ignite ignite = grid(0);
+
+        final String key = "key";
+
+        Lock lock = ignite.cache(null).lock(key);
+
+        lock.lock();
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite.createCache("NEW_CACHE");
+
+                return null;
+            }
+        }, IgniteException.class, "Cannot start/stop cache within lock.");
+
+        lock.unlock();
+    }
 }