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

[1/7] incubator-ignite git commit: #ignite-994: inject ignite before using.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-gg-10404 32697ab08 -> c62c410bc


#ignite-994: inject ignite before using.


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

Branch: refs/heads/ignite-gg-10404
Commit: 5db097342411abe6dbdd3d1f9c70029b3b497aff
Parents: 8f455a9
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 10 15:38:17 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 10 15:38:17 2015 +0300

----------------------------------------------------------------------
 .../continuous/GridContinuousProcessor.java          | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5db09734/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 38d970b..dd04bf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -343,16 +343,19 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         if (!ctx.isDaemon() && data != null) {
             for (DiscoveryDataItem item : data.items) {
-                // Register handler only if local node passes projection predicate.
-                if (item.prjPred == null || item.prjPred.apply(ctx.discovery().localNode())) {
-                    try {
+                try {
+                    if (item.prjPred != null)
+                        ctx.resource().injectGeneric(item.prjPred);
+
+                    // Register handler only if local node passes projection predicate.
+                    if (item.prjPred == null || item.prjPred.apply(ctx.discovery().localNode())) {
                         if (registerHandler(data.nodeId, item.routineId, item.hnd, item.bufSize, item.interval,
                             item.autoUnsubscribe, false))
                             item.hnd.onListenerRegistered(item.routineId, ctx);
                     }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to register continuous handler.", e);
-                    }
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to register continuous handler.", e);
                 }
             }
 


[3/7] incubator-ignite git commit: #ignite-1008: small change.

Posted by ak...@apache.org.
#ignite-1008: small change.


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

Branch: refs/heads/ignite-gg-10404
Commit: fb8b9963fbe25a36f860a31ec6a66bfecd035cbe
Parents: 0be4bec
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 22:47:06 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 22:47:06 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java      | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fb8b9963/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 28341ad..4fdec33 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
@@ -2956,8 +2956,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (transactions().tx() != null)
             throw new IgniteException("Cannot start/stop cache within transaction.");
 
-        if (ctx.cache().context().mvcc().
-            lastExplicitLockTopologyVersion(Thread.currentThread().getId()) != null)
+        if (sharedCtx.mvcc().lastExplicitLockTopologyVersion(Thread.currentThread().getId()) != null)
             throw new IgniteException("Cannot start/stop cache within lock.");
     }
 


[5/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-994' into ignite-sprint-6

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-994' into ignite-sprint-6


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

Branch: refs/heads/ignite-gg-10404
Commit: b087aca99243abb8569eecd572ada1414bfd7ce0
Parents: 904888b 5db0973
Author: sboikov <se...@inria.fr>
Authored: Fri Jun 12 07:50:25 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri Jun 12 07:50:25 2015 +0300

----------------------------------------------------------------------
 .../continuous/GridContinuousProcessor.java          | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[2/7] incubator-ignite git commit: #ignite-1008: Add check for transactions in cache start/stop.

Posted by ak...@apache.org.
#ignite-1008: Add check for transactions in cache start/stop.


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

Branch: refs/heads/ignite-gg-10404
Commit: 0be4becc99be9db8de840f0502703e278eb9f5b1
Parents: 2b63ff8
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jun 11 22:33:25 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jun 11 22:34:37 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  18 ++
 ...teStartCacheInTransactionAtomicSelfTest.java |  32 +++
 .../IgniteStartCacheInTransactionSelfTest.java  | 254 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 4 files changed, 307 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0be4becc/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 9b16388..28341ad 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
@@ -1874,6 +1874,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         CacheType cacheType,
         boolean failIfExists
     ) {
+        checkEmptyTransactions();
+
         assert ccfg != null || nearCfg != null;
 
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
@@ -1964,6 +1966,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));
@@ -2647,6 +2651,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);
@@ -2944,6 +2950,18 @@ 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.");
+
+        if (ctx.cache().context().mvcc().
+            lastExplicitLockTopologyVersion(Thread.currentThread().getId()) != null)
+            throw new IgniteException("Cannot start/stop cache within lock.");
+    }
+
+    /**
      * @param val Object to check.
      * @throws IgniteCheckedException If validation failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0be4becc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionAtomicSelfTest.java
new file mode 100644
index 0000000..17320eb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionAtomicSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Check starting cache in transaction.
+ */
+public class IgniteStartCacheInTransactionAtomicSelfTest extends IgniteStartCacheInTransactionSelfTest {
+    /** {@inheritDoc} */
+    @Override public CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0be4becc/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..fb0bee2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * 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(atomicityMode());
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    public CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    public CacheConfiguration cacheConfiguration(String cacheName) {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setName(cacheName);
+
+        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 testStartConfigurationCache() 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(cacheConfiguration("NEW_CACHE"));
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartConfigurationCacheWithNear() 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(cacheConfiguration("NEW_CACHE"), new NearCacheConfiguration());
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOrCreateCache() 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.getOrCreateCache("NEW_CACHE");
+
+                    return null;
+                }
+            }, IgniteException.class, "Cannot start/stop cache within transaction.");
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOrCreateCacheConfiguration() 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.getOrCreateCache(cacheConfiguration("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 {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        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/0be4becc/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 a503e1e..c598e38 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
@@ -138,6 +138,9 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(IgniteCacheManyClientsTest.class);
 
+        suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
+        suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);
+
         return suite;
     }
 }


[6/7] incubator-ignite git commit: Merge branches 'ignite-gg-10404' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404

Posted by ak...@apache.org.
Merge branches 'ignite-gg-10404' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404


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

Branch: refs/heads/ignite-gg-10404
Commit: 67790714098c9c413a06b855c5b51e665a39e755
Parents: 32697ab b087aca
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 12 14:15:58 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 12 14:15:58 2015 +0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  17 ++
 .../continuous/GridContinuousProcessor.java     |  15 +-
 ...teStartCacheInTransactionAtomicSelfTest.java |  32 +++
 .../IgniteStartCacheInTransactionSelfTest.java  | 254 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 5 files changed, 315 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[4/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1008-v2' into ignite-sprint-6

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1008-v2' into ignite-sprint-6


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

Branch: refs/heads/ignite-gg-10404
Commit: 904888b2b468637c76075523eee68506fef7c28e
Parents: f0e74c0 fb8b996
Author: sboikov <se...@inria.fr>
Authored: Fri Jun 12 07:44:41 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri Jun 12 07:44:41 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  17 ++
 ...teStartCacheInTransactionAtomicSelfTest.java |  32 +++
 .../IgniteStartCacheInTransactionSelfTest.java  | 254 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 4 files changed, 306 insertions(+)
----------------------------------------------------------------------



[7/7] incubator-ignite git commit: # GG-10404 Suppress Visor exceptions.

Posted by ak...@apache.org.
# GG-10404 Suppress Visor exceptions.


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

Branch: refs/heads/ignite-gg-10404
Commit: c62c410bcf1a1f353fa586e515f846746cb4a482
Parents: 6779071
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 12 17:39:09 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 12 17:39:09 2015 +0700

----------------------------------------------------------------------
 .../connection/GridClientNioTcpConnection.java  |  7 +----
 .../processors/rest/GridRestProcessor.java      |  4 ++-
 .../handlers/task/GridTaskCommandHandler.java   | 12 ++++---
 .../processors/task/GridTaskWorker.java         |  4 ++-
 .../visor/query/VisorQueryCleanupTask.java      |  4 +--
 .../util/VisorClusterGroupEmptyException.java   | 33 ++++++++++++++++++++
 .../visor/util/VisorEmptyTopologyException.java | 33 --------------------
 7 files changed, 50 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index 67709b8..d247e05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -750,12 +750,7 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             new GridClientFutureCallback<GridClientTaskResultBean, R>() {
                 @Override public R onComplete(GridClientFuture<GridClientTaskResultBean> fut)
                     throws GridClientException {
-                    GridClientTaskResultBean resBean = fut.get();
-
-                    if (resBean != null)
-                        return resBean.getResult();
-                    else
-                        return null;
+                    return fut.get().getResult();
                 }
             });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 52ca610..2d1d802 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.plugin.security.SecurityException;
@@ -214,7 +215,8 @@ public class GridRestProcessor extends GridProcessorAdapter {
                     res = f.get();
                 }
                 catch (Exception e) {
-                    LT.error(log, e, "Failed to handle request: " + req.command());
+                    if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                        LT.error(log, e, "Failed to handle request: " + req.command());
 
                     if (log.isDebugEnabled())
                         log.debug("Failed to handle request [req=" + req + ", e=" + e + "]");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
index a647cd1..d832b21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.jetbrains.annotations.*;
@@ -134,7 +135,8 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
             return handleAsyncUnsafe(req);
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to execute task command: " + req, e);
+            if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                U.error(log, "Failed to execute task command: " + req, e);
 
             return new GridFinishedFuture<>(e);
         }
@@ -237,9 +239,11 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                                     U.warn(log, "Failed to execute task due to topology issues (are all mapped " +
                                         "nodes alive?) [name=" + name + ", clientId=" + req.clientId() +
                                         ", err=" + e + ']');
-                                else
-                                    U.error(log, "Failed to execute task [name=" + name + ", clientId=" +
-                                        req.clientId() + ']', e);
+                                else {
+                                    if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                                        U.error(log, "Failed to execute task [name=" + name + ", clientId=" +
+                                            req.clientId() + ']', e);
+                                }
 
                                 desc = new TaskDescriptor(true, null, e);
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index f6d686c..eb5fa77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.resources.*;
@@ -443,7 +444,8 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         }
         catch (IgniteException | IgniteCheckedException e) {
             if (!fut.isCancelled()) {
-                U.error(log, "Failed to map task jobs to nodes: " + ses, e);
+                if (!(e instanceof VisorClusterGroupEmptyException))
+                    U.error(log, "Failed to map task jobs to nodes: " + ses, e);
 
                 finishTask(null, e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
index b9a55e1..5ceb300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
@@ -49,7 +49,7 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
         Set<UUID> nodeIds = taskArg.keySet();
 
         if (nodeIds.isEmpty())
-            throw new VisorEmptyTopologyException("Nothing to clear. List with node IDs is empty!");
+            throw new VisorClusterGroupEmptyException("Nothing to clear. List with node IDs is empty!");
 
         Map<ComputeJob, ClusterNode> map = U.newHashMap(nodeIds.size());
 
@@ -64,7 +64,7 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
                 for (UUID nid : nodeIds)
                     notFoundNodes = notFoundNodes + (notFoundNodes.isEmpty() ? "" : ",")  + U.id8(nid);
 
-                throw new VisorEmptyTopologyException("Failed to clear query results. Nodes are not available: [" +
+                throw new VisorClusterGroupEmptyException("Failed to clear query results. Nodes are not available: [" +
                     notFoundNodes + "]");
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
new file mode 100644
index 0000000..b969178
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.visor.util;
+
+import org.apache.ignite.cluster.*;
+
+/**
+ * Exception to throw from Visor tasks in case of empty topology.
+ */
+public class VisorClusterGroupEmptyException extends ClusterGroupEmptyException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** @inheritDoc */
+    public VisorClusterGroupEmptyException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
deleted file mode 100644
index fda1bd7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
+++ /dev/null
@@ -1,33 +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.visor.util;
-
-import org.apache.ignite.*;
-
-/**
- * Marker exception for indication of empty topology in Visor tasks.
- */
-public class VisorEmptyTopologyException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** @inheritDoc */
-    public VisorEmptyTopologyException(String msg) {
-        super(msg);
-    }
-}