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/22 19:02:19 UTC

[1/6] incubator-ignite git commit: # ignite-648: move isMultiJvm

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 63e98328c -> 470e48b54


# ignite-648: move isMultiJvm


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

Branch: refs/heads/ignite-648
Commit: 2e8c0fef91f05aa6ad61509aee35b4a151ce1264
Parents: 63e9832
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 13:24:49 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 13:24:49 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractSelfTest.java        | 25 --------------------
 .../IpcSharedMemoryNativeLoaderSelfTest.java    |  1 +
 .../testframework/junits/GridAbstractTest.java  | 25 ++++++++++++++++++++
 3 files changed, 26 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e8c0fef/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index f129dd0..b6e56d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -358,31 +358,6 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Gets flag whether nodes will run in one jvm or in separate jvms.
-     *
-     * @return <code>True</code> to run nodes in separate jvms.
-     */
-    protected boolean isMultiJvm() {
-        return false;
-    }
-
-    /**
-     * @param gridName Grid name.
-     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
-     */
-    protected boolean isMultiJvmAndNodeIsRemote(String gridName) {
-        return isMultiJvm() && !gridName.endsWith("0");
-    }
-
-    /**
-     * @param idx Grid index.
-     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
-     */
-    protected boolean isMultiJvmAndNodeIsRemote(int idx) {
-        return isMultiJvm() && idx != 0;
-    }
-
-    /**
      * @param idx Index of grid.
      * @return Cache context.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e8c0fef/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoaderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoaderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoaderSelfTest.java
index 6a72698..65fa543 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoaderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoaderSelfTest.java
@@ -28,6 +28,7 @@ import java.util.*;
  * Test shared memory native loader.
  */
 public class IpcSharedMemoryNativeLoaderSelfTest extends TestCase {
+
     /**
      * Test {@link IpcSharedMemoryNativeLoader#load()} in case, when native library path was
      * already loaded, but corrupted.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e8c0fef/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 9c42920..fb5ca51 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -1286,6 +1286,31 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * Gets flag whether nodes will run in one jvm or in separate jvms.
+     *
+     * @return <code>True</code> to run nodes in separate jvms.
+     */
+    protected boolean isMultiJvm() {
+        return false;
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
+     */
+    protected boolean isMultiJvmAndNodeIsRemote(String gridName) {
+        return isMultiJvm() && !gridName.endsWith("0");
+    }
+
+    /**
+     * @param idx Grid index.
+     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
+     */
+    protected boolean isMultiJvmAndNodeIsRemote(int idx) {
+        return isMultiJvm() && idx != 0;
+    }
+
+    /**
      * @return Test counters.
      */
     protected synchronized TestCounters getTestCounters() throws IgniteCheckedException {


[3/6] incubator-ignite git commit: # ignite-648: AffinityProcessProxy and fix another process stopping

Posted by sb...@apache.org.
# ignite-648: AffinityProcessProxy and fix another process stopping


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

Branch: refs/heads/ignite-648
Commit: 9cdd165ba3d4e372f4ec17c68e9a951b892000e4
Parents: 9ed8a4e
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 16:52:12 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 16:52:12 2015 +0300

----------------------------------------------------------------------
 .../framework/AffinityProcessProxy.java         | 197 +++++++++++++++++++
 .../multijvm/framework/IgniteProcessProxy.java  |   9 +-
 .../testframework/junits/GridAbstractTest.java  |   4 +-
 3 files changed, 204 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cdd165b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
new file mode 100644
index 0000000..07a5a5f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
@@ -0,0 +1,197 @@
+/*
+ * 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.multijvm.framework;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Proxy class for affinity at another jvm.
+ */
+public class AffinityProcessProxy<K> implements Affinity<K> {
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param cacheName Cache name.
+     * @param proxy Ignite ptocess proxy.
+     */
+    public AffinityProcessProxy(String cacheName, IgniteProcessProxy proxy) {
+        this.cacheName = cacheName;
+        gridId = proxy.getId();
+
+        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
+
+        compute = proxy.localJvmGrid().compute(grp);
+    }
+
+    /**
+     * Returns cache instance. Method to be called from closure at another JVM.
+     *
+     * @return Cache.
+     */
+    private Affinity<Object> affinity() {
+        return Ignition.ignite(gridId).affinity(cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partitions() {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().partitions();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(final K key) {
+        return (int)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().partition(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPrimary(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isPrimary(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isBackup(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isBackup(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPrimaryOrBackup(final ClusterNode n, final K key) {
+        return (boolean)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().isPrimaryOrBackup(n, key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] primaryPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().primaryPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] backupPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().backupPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] allPartitions(final ClusterNode n) {
+        return (int[])compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().allPartitions(n);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object affinityKey(final K key) {
+        return compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().affinityKey(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<ClusterNode, Collection<K>> mapKeysToNodes(final Collection<? extends K> keys) {
+        return (Map<ClusterNode, Collection<K>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeysToNodes(keys);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode mapKeyToNode(final K key) {
+        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeyToNode(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> mapKeyToPrimaryAndBackups(final K key) {
+        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapKeyToPrimaryAndBackups(key);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode mapPartitionToNode(final int part) {
+        return (ClusterNode)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionToNode(part);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<Integer, ClusterNode> mapPartitionsToNodes(final Collection<Integer> parts) {
+        return (Map<Integer, ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionsToNodes(parts);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> mapPartitionToPrimaryAndBackups(final int part) {
+        return (Collection<ClusterNode>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                return affinity().mapPartitionToPrimaryAndBackups(part);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cdd165b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
index 71c6689..95ce583 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
@@ -137,17 +137,16 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /**
      * Kill all running processes.
-     *
-     * @throws Exception if failed.
      */
-    public static void killAll() throws Exception {
-        for (IgniteProcessProxy ignite : gridProxies.values())
+    public static void killAll() {
+        for (IgniteProcessProxy ignite : gridProxies.values()) {
             try {
                 ignite.getProcess().kill();
             }
             catch (Exception e) {
                 U.error(ignite.log, "Killing failed.", e);
             }
+        }
 
         gridProxies.clear();
     }
@@ -450,7 +449,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K> Affinity<K> affinity(String cacheName) {
-        return null; // TODO: CODE: implement.
+        return new AffinityProcessProxy(cacheName, this);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cdd165b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 990ea76..b67f5b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -441,7 +441,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected void afterTestsStopped() throws Exception {
-        IgniteProcessProxy.killAll();
+        // No-op.
     }
 
     /** {@inheritDoc} */
@@ -775,6 +775,8 @@ public abstract class GridAbstractTest extends TestCase {
      * @param cancel Cancel flag.
      */
     protected void stopAllGrids(boolean cancel) {
+        IgniteProcessProxy.killAll(); // In multi jvm case.
+
         Collection<Ignite> clients = new ArrayList<>();
         Collection<Ignite> srvs = new ArrayList<>();
 


[2/6] incubator-ignite git commit: # ignite-648: get vs grid methods (fix some tests automatically)

Posted by sb...@apache.org.
# ignite-648: get vs grid methods (fix some tests automatically)


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

Branch: refs/heads/ignite-648
Commit: 9ed8a4e627256a7add543326b24b42ae4b39050f
Parents: 2e8c0fe
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 15:57:27 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 15:57:27 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 73 ++----------------
 .../cache/GridCacheAbstractSelfTest.java        | 13 ++--
 .../multijvm/framework/IgniteProcessProxy.java  | 25 +++----
 .../testframework/junits/GridAbstractTest.java  | 78 +++++++++++++++++++-
 4 files changed, 96 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 9314b70..292773c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -27,7 +27,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.affinity.*;
-import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -35,8 +34,6 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -65,11 +62,6 @@ import static org.apache.ignite.transactions.TransactionState.*;
  * Full API cache test.
  */
 public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
-    /** Ip finder for TCP discovery. */
-    public static final TcpDiscoveryIpFinder LOCAL_IP_FINDER = new TcpDiscoveryVmIpFinder(false) {{
-        setAddresses(Collections.singleton("127.0.0.1:47500..47509"));
-    }};
-
     /** Increment processor for invoke operations. */
     public static final EntryProcessor<String, Integer, String> INCR_PROCESSOR = new EntryProcessor<String, Integer, String>() {
         @Override public String process(MutableEntry<String, Integer> e, Object... args) {
@@ -118,18 +110,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /** */
     private Map<String, CacheConfiguration[]> cacheCfgMap;
 
-    /** All nodes join latch (for multi JVM mode). */
-    private CountDownLatch allNodesJoinLatch;
-
-    /** Node join listener (for multi JVM mode). */
-    private final IgnitePredicate<Event> nodeJoinLsnr = new IgnitePredicate<Event>() {
-        @Override public boolean apply(Event evt) {
-            allNodesJoinLatch.countDown();
-
-            return true;
-        }
-    };
-
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 1;
@@ -156,14 +136,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES)
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 
-        if (isMultiJvm()) {
-            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(LOCAL_IP_FINDER);
-
-            cfg.setLocalEventListeners(new HashMap<IgnitePredicate<? extends Event>, int[]>() {{
-                put(nodeJoinLsnr, new int[] {EventType.EVT_NODE_JOINED});
-            }});
-        }
-
         return cfg;
     }
 
@@ -180,10 +152,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /** {@inheritDoc} */
+    // TODO review. See IgnitionEx.grid(entry.getKey()).
     @Override protected void beforeTestsStarted() throws Exception {
-        if (isMultiJvm())
-            allNodesJoinLatch = new CountDownLatch(gridCount() - 1);
-
         if (cacheStartType() == CacheStartMode.STATIC)
             super.beforeTestsStarted();
         else {
@@ -194,11 +164,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
                 for (Map.Entry<String, CacheConfiguration[]> entry : cacheCfgMap.entrySet()) {
                     Ignite ignite;
-
-                    if (isMultiJvm())
-                        ignite = IgniteProcessProxy.grid(entry.getKey());
-                    else
-                        ignite = IgnitionEx.grid(entry.getKey());
+                    
+                    ignite = IgnitionEx.grid(entry.getKey());
 
                     for (CacheConfiguration cfg : entry.getValue())
                         ignite.createCache(cfg);
@@ -229,21 +196,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++)
             info("Grid " + i + ": " + grid(i).localNode().id());
-
-        if (isMultiJvm())
-            assert allNodesJoinLatch.await(5, TimeUnit.SECONDS);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        IgniteProcessProxy.killAll();
-
-        super.afterTestsStopped();
     }
 
     /** {@inheritDoc} */
     @Override protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
-        if (!isMultiJvm() || gridName.endsWith("0")) {
+        if (!isMultiJvmAndNodeIsRemote(gridName)) {
             if (cacheCfgMap == null)
                 return super.startGrid(gridName, ctx);
 
@@ -256,27 +213,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             return IgnitionEx.start(optimize(cfg), ctx);
         }
 
-        return new IgniteProcessProxy(optimize(getConfiguration(gridName)), log, grid(0));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteEx grid(int idx) {
-        if (!isMultiJvm() || idx == 0)
-            return super.grid(idx);
-
-        return IgniteProcessProxy.get(getTestGridName(idx));
-    }
-
-    /**
-     * @param idx Index of grid.
-     * @return Default cache.
-     */
-    @SuppressWarnings({"unchecked"})
-    @Override protected IgniteCache<String, Integer> jcache(int idx) {
-        if (!isMultiJvm() || idx == 0)
-            return super.jcache(idx);
-
-        return IgniteProcessProxy.get(getTestGridName(idx)).cache(null);
+        return startRemoteGrid(gridName, ctx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index b6e56d5..9168c67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -60,11 +60,6 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     /** VM ip finder for TCP discovery. */
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
-    /**
-     * @return Grids count to start.
-     */
-    protected abstract int gridCount();
-
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
         return TEST_TIMEOUT;
@@ -354,7 +349,11 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked"})
     @Override protected IgniteCache<String, Integer> jcache(int idx) {
-        return ignite(idx).cache(null);
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return ignite(idx).cache(null);
+        else
+            return IgniteProcessProxy.get(getTestGridName(idx)).cache(null);
+
     }
 
     /**
@@ -367,7 +366,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
         else {
 //            ((IgniteProcessProxy)grid(idx)).remoteInternalCache();
 
-            // TODO
+            // TODO refix it.
             final UUID id = ((IgniteProcessProxy)grid(idx)).getId();
 
             return new GridCacheContext<String, Integer>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
index 538f316..71c6689 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
@@ -123,10 +123,16 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /**
      * @param gridName Grid name.
-     * @return Instance by name or <code>null</code>.
+     * @return Instance by name or exception wiil be thrown.
      */
     public static IgniteProcessProxy get(String gridName) {
-        return gridProxies.get(gridName);
+        IgniteProcessProxy res = gridProxies.get(gridName);
+
+        if (res == null)
+            throw new IgniteIllegalStateException("Grid instance was not properly started " +
+                "or was already stopped: " + gridName);
+
+        return res;
     }
 
     /**
@@ -447,22 +453,13 @@ public class IgniteProcessProxy implements IgniteEx {
         return null; // TODO: CODE: implement.
     }
 
+    /**
+     * @return Jvm process in which grid node started.
+     */
     public GridJavaProcess getProcess() {
         return proc;
     }
 
-    public static Ignite grid(@Nullable String name) {
-//        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        Ignite res = gridProxies.get(name);
-
-        if (res == null)
-            throw new IgniteIllegalStateException("Grid instance was not properly started " +
-                "or was already stopped: " + name);
-
-        return res;
-    }
-
     // TODO delete or use.
 //    public <K, V> GridCacheAdapter<K, V> remoteInternalCache() {
 //        return (GridCacheAdapter<K, V>)compute.call(new MyCallable(id));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index fb5ca51..990ea76 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -33,7 +34,9 @@ import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.spi.checkpoint.sharedfs.*;
 import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.logger.*;
@@ -72,6 +75,11 @@ public abstract class GridAbstractTest extends TestCase {
     /** Null name for execution map. */
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
+    /** Ip finder for TCP discovery. */
+    public static final TcpDiscoveryIpFinder LOCAL_IP_FINDER = new TcpDiscoveryVmIpFinder(false) {{
+        setAddresses(Collections.singleton("127.0.0.1:47500..47509"));
+    }};
+
     /** */
     private static final long DFLT_TEST_TIMEOUT = 5 * 60 * 1000;
 
@@ -96,6 +104,18 @@ public abstract class GridAbstractTest extends TestCase {
     /** Starting grid name. */
     protected static ThreadLocal<String> startingGrid = new ThreadLocal<>();
 
+    /** All nodes join latch (for multi JVM mode). */
+    private CountDownLatch allNodesJoinLatch;
+
+    /** Node join listener (for multi JVM mode). */
+    private final IgnitePredicate<Event> nodeJoinLsnr = new IgnitePredicate<Event>() {
+        @Override public boolean apply(Event evt) {
+            allNodesJoinLatch.countDown();
+
+            return true;
+        }
+    };
+
     /**
      *
      */
@@ -421,7 +441,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected void afterTestsStopped() throws Exception {
-        // No-op.
+        IgniteProcessProxy.killAll();
     }
 
     /** {@inheritDoc} */
@@ -464,7 +484,13 @@ public abstract class GridAbstractTest extends TestCase {
             }
 
             try {
+                if (isMultiJvm() && gridCount() > 0)
+                    allNodesJoinLatch = new CountDownLatch(gridCount() - 1);
+
                 beforeTestsStarted();
+
+                if (isMultiJvm() && gridCount() > 0)
+                    assert allNodesJoinLatch.await(5, TimeUnit.SECONDS);
             }
             catch (Exception | Error t) {
                 t.printStackTrace();
@@ -668,6 +694,19 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * Starts new grid at another JVM with given name.
+     *
+     * @param gridName Grid name.
+     * @param ctx Spring context.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
+    // TODO review. Is it okey that ctx doesn't used?
+    protected Ignite startRemoteGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
+        return new IgniteProcessProxy(optimize(getConfiguration(gridName)), log, grid(0));
+    }
+
+    /**
      * Optimizes configuration to achieve better test performance.
      *
      * @param cfg Configuration.
@@ -821,7 +860,10 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid instance.
      */
     protected IgniteEx grid(String name) {
-        return (IgniteEx)G.ignite(name);
+        if (!isMultiJvmAndNodeIsRemote(name))
+            return (IgniteEx)G.ignite(name);
+        else
+            return IgniteProcessProxy.get(name);
     }
 
     /**
@@ -831,7 +873,11 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid instance.
      */
     protected IgniteEx grid(int idx) {
-        return (IgniteEx)G.ignite(getTestGridName(idx));
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return (IgniteEx)G.ignite(getTestGridName(idx));
+        else
+            return IgniteProcessProxy.get(getTestGridName(idx));
+
     }
 
     /**
@@ -839,7 +885,12 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Ignite instance.
      */
     protected Ignite ignite(int idx) {
-        return G.ignite(getTestGridName(idx));
+        String gridName = getTestGridName(idx);
+
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return G.ignite(gridName);
+        else
+            return IgniteProcessProxy.get(gridName);
     }
 
     /**
@@ -847,6 +898,7 @@ public abstract class GridAbstractTest extends TestCase {
      *
      * @return Grid for given test.
      */
+    // TODO isMultyJvm.
     protected IgniteEx grid() {
         return (IgniteEx)G.ignite(getTestGridName());
     }
@@ -855,6 +907,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @param node Node.
      * @return Ignite instance with given local node.
      */
+    // TODO isMultyJvm.
     protected final Ignite grid(ClusterNode node) {
         return G.ignite(node.id());
     }
@@ -869,6 +922,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid Started grid.
      * @throws Exception If failed.
      */
+    // TODO isMultyJvm.
     protected Ignite startGrid(String gridName, String springCfgPath) throws Exception {
         return startGrid(gridName, loadConfiguration(springCfgPath));
     }
@@ -883,6 +937,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid Started grid.
      * @throws Exception If failed.
      */
+    // TODO isMultyJvm.
     protected Ignite startGrid(String gridName, IgniteConfiguration cfg) throws Exception {
         cfg.setGridName(gridName);
 
@@ -1015,6 +1070,14 @@ public abstract class GridAbstractTest extends TestCase {
             cfg.setNodeId(UUID.fromString(new String(chars)));
         }
 
+        if (isMultiJvm()) {
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(LOCAL_IP_FINDER);
+
+            cfg.setLocalEventListeners(new HashMap<IgnitePredicate<? extends Event>, int[]>() {{
+                put(nodeJoinLsnr, new int[] {EventType.EVT_NODE_JOINED});
+            }});
+        }
+
         return cfg;
     }
 
@@ -1286,6 +1349,13 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @return Grids count to start.
+     */
+    protected int gridCount() {
+        return 0;
+    }
+
+    /**
      * Gets flag whether nodes will run in one jvm or in separate jvms.
      *
      * @return <code>True</code> to run nodes in separate jvms.


[6/6] incubator-ignite git commit: # ignite-648: cache proxy; add events proxy; delete transaction support for multiJvm; skip some tests for multi jvm

Posted by sb...@apache.org.
# ignite-648: cache proxy; add events proxy; delete transaction support for multiJvm; skip some tests for multi jvm


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

Branch: refs/heads/ignite-648
Commit: 470e48b54debab29b7fd1fc39e306876ae379e50
Parents: 2e6fd9b
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 20:02:33 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 20:02:33 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java |  64 +++++---
 .../framework/AffinityProcessProxy.java         |   5 +-
 .../framework/IgniteCacheProcessProxy.java      |  43 ++++--
 .../framework/IgniteEventsProcessProxy.java     | 147 +++++++++++++++++++
 .../multijvm/framework/IgniteProcessProxy.java  |  16 +-
 .../IgniteTransactionsProcessProxy.java         |  78 ----------
 6 files changed, 237 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 75cba54..85eb08e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -105,7 +105,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         };
 
     /** Dflt grid. */
-    protected Ignite dfltIgnite;
+    protected transient Ignite dfltIgnite;
 
     /** */
     private Map<String, CacheConfiguration[]> cacheCfgMap;
@@ -291,15 +291,20 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         }
 
         for (int i = 0; i < gridCount(); i++) {
-            GridCacheContext<String, Integer> ctx = context(i);
+            if (!isMultiJvmAndNodeIsRemote(i)) {
+                GridCacheContext<String, Integer> ctx = context(i);
 
-            int sum = 0;
+                int sum = 0;
 
-            for (String key : map.keySet())
-                if (ctx.affinity().localNode(key, new AffinityTopologyVersion(ctx.discovery().topologyVersion())))
-                    sum++;
+                for (String key : map.keySet())
+                    if (ctx.affinity().localNode(key, new AffinityTopologyVersion(ctx.discovery().topologyVersion())))
+                        sum++;
 
-            assertEquals("Incorrect key size on cache #" + i, sum, jcache(i).localSize(ALL));
+                assertEquals("Incorrect key size on cache #" + i, sum, jcache(i).localSize(ALL));
+            }
+            else {
+                // TODO add multi jvm support.
+            }
         }
 
         for (int i = 0; i < gridCount(); i++) {
@@ -2949,14 +2954,16 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testPeekTxRemoveOptimistic() throws Exception {
-        checkPeekTxRemove(OPTIMISTIC);
+        if (!isMultiJvm()) // Transactions are not supported in multi JVM mode.
+            checkPeekTxRemove(OPTIMISTIC);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPeekTxRemovePessimistic() throws Exception {
-        checkPeekTxRemove(PESSIMISTIC);
+        if (!isMultiJvm()) // Transactions are not supported in multi JVM mode.
+            checkPeekTxRemove(PESSIMISTIC);
     }
 
     /**
@@ -3098,7 +3105,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlTx() throws Exception {
-        if (txEnabled())
+        if (txEnabled() && !isMultiJvm())
             checkTtl(true, false);
     }
 
@@ -3106,6 +3113,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlNoTx() throws Exception {
+        if (isMultiJvm())
+            fail("TODO implement multi jvm support.");
+
         checkTtl(false, false);
     }
 
@@ -3113,6 +3123,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlNoTxOldEntry() throws Exception {
+        if (isMultiJvm())
+            fail("TODO implement multi jvm support.");
+
         checkTtl(false, true);
     }
 
@@ -3805,26 +3818,31 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             assertEquals(keys.size(), jcache().localSize(CachePeekMode.ALL));
         else {
             for (int i = 0; i < gridCount(); i++) {
-                GridCacheContext<String, Integer> ctx = context(i);
+                if (!isMultiJvmAndNodeIsRemote(i)) {
+                    GridCacheContext<String, Integer> ctx = context(i);
 
-                if (ctx.cache().configuration().getMemoryMode() == OFFHEAP_TIERED)
-                    continue;
+                    if (ctx.cache().configuration().getMemoryMode() == OFFHEAP_TIERED)
+                        continue;
 
-                int size = 0;
+                    int size = 0;
 
-                for (String key : keys) {
-                    if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) {
-                        GridCacheEntryEx e =
-                            ctx.isNear() ? ctx.near().dht().peekEx(key) : ctx.cache().peekEx(key);
+                    for (String key : keys) {
+                        if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) {
+                            GridCacheEntryEx e =
+                                ctx.isNear() ? ctx.near().dht().peekEx(key) : ctx.cache().peekEx(key);
 
-                        assert e != null : "Entry is null [idx=" + i + ", key=" + key + ", ctx=" + ctx + ']';
-                        assert !e.deleted() : "Entry is deleted: " + e;
+                            assert e != null : "Entry is null [idx=" + i + ", key=" + key + ", ctx=" + ctx + ']';
+                            assert !e.deleted() : "Entry is deleted: " + e;
 
-                        size++;
+                            size++;
+                        }
                     }
-                }
 
-                assertEquals("Incorrect size on cache #" + i, size, jcache(i).localSize(ALL));
+                    assertEquals("Incorrect size on cache #" + i, size, jcache(i).localSize(ALL));
+                }
+                else {
+                    // TODO add multi jvm support.
+                }
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
index 07a5a5f..7168534 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/AffinityProcessProxy.java
@@ -45,10 +45,7 @@ public class AffinityProcessProxy<K> implements Affinity<K> {
     public AffinityProcessProxy(String cacheName, IgniteProcessProxy proxy) {
         this.cacheName = cacheName;
         gridId = proxy.getId();
-
-        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
-
-        compute = proxy.localJvmGrid().compute(grp);
+        compute = proxy.remoteCompute();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
index 61cbdc3..f8cd8a3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
@@ -71,10 +71,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
         isAsync = async;
         gridId = proxy.getId();
         igniteProxy = proxy;
-
-        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
-
-        compute = proxy.localJvmGrid().compute(grp);
+        compute = proxy.remoteCompute();
     }
 
     /** {@inheritDoc} */
@@ -89,6 +86,15 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
     /** {@inheritDoc} */
     @Override public <R> IgniteFuture<R> future() {
+        // TODO implement.
+//        R futureRes = (R)compute.call(new IgniteCallable<Object>() {
+//            @Override public Object call() throws Exception {
+//                return cache().future().get();
+//            }
+//        });
+//
+//        return new IgniteFinishedFutureImpl<R>(futureRes);
+
         throw new UnsupportedOperationException("Method should be supported.");
     }
 
@@ -124,8 +130,14 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        throw new UnsupportedOperationException("Method should be supported.");
+    @Override public void localLoadCache(@Nullable final IgniteBiPredicate<K, V> p, @Nullable final Object... args) throws CacheException {
+        final IgniteBiPredicate pCopy = p;
+
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().localLoadCache(pCopy, args);
+            }
+        });
     }
 
     /** {@inheritDoc} */
@@ -178,8 +190,12 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void localEvict(Collection<? extends K> keys) {
-        throw new UnsupportedOperationException("Method should be supported.");
+    @Override public void localEvict(final Collection<? extends K> keys) {
+        compute.run(new IgniteRunnable() {
+            @Override public void run() {
+                cache().localEvict(keys);
+            }
+        });
     }
 
     /** {@inheritDoc} */
@@ -501,7 +517,16 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
     /** {@inheritDoc} */
     @Override public <T> T unwrap(final Class<T> clazz) {
-        throw new UnsupportedOperationException("Method cannot be supported because T can be unmarshallable.");
+        try {
+            return (T)compute.call(new IgniteCallable<Object>() {
+                @Override public Object call() throws Exception {
+                    return cache().unwrap(clazz);
+                }
+            });
+        }
+        catch (Exception e) {
+            throw new IllegalArgumentException("Looks like class " + clazz + " is unmarshallable. Exception type:" + e.getClass(), e);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
new file mode 100644
index 0000000..b9a260b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteEventsProcessProxy.java
@@ -0,0 +1,147 @@
+/*
+ * 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.multijvm.framework;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Ignite events proxy for ignite instance at another JVM.
+ */
+public class IgniteEventsProcessProxy implements IgniteEvents {
+    /** Ignite proxy. */
+    private final transient IgniteProcessProxy igniteProxy;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param igniteProxy Ignite proxy.
+     */
+    public IgniteEventsProcessProxy(IgniteProcessProxy igniteProxy) {
+        this.igniteProxy = igniteProxy;
+
+        gridId = igniteProxy.getId();
+    }
+
+    /**
+     * @return Events instance.
+     */
+    private IgniteEvents events() {
+        return Ignition.ignite(gridId).events();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup clusterGroup() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
+        @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> UUID remoteListen(@Nullable IgniteBiPredicate<UUID, T> locLsnr,
+        @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> UUID remoteListen(int bufSize, long interval, boolean autoUnsubscribe,
+        @Nullable IgniteBiPredicate<UUID, T> locLsnr, @Nullable IgnitePredicate<T> rmtFilter,
+        @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopRemoteListen(UUID opId) throws IgniteException {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
+        @Nullable int... types) throws IgniteException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void recordLocal(Event evt) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void localListen(final IgnitePredicate<? extends Event> lsnr, final int... types) {
+        igniteProxy.remoteCompute().run(new IgniteRunnable() {
+            @Override public void run() {
+                events().localListen(lsnr, types);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void enableLocal(int... types) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disableLocal(int... types) {
+        // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] enabledEvents() {
+        return new int[0]; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEnabled(int type) {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteEvents withAsync() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        return false; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        return null; // TODO: CODE: implement.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
index 1d11dc3..e534478 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
@@ -277,7 +277,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteEvents events() {
-        return null; // TODO: CODE: implement.
+        return new IgniteEventsProcessProxy(this);
     }
 
     /** {@inheritDoc} */
@@ -375,7 +375,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
-        return new IgniteTransactionsProcessProxy(this);
+        throw new UnsupportedOperationException("Transactions are not supported in multi JVM mode.");
     }
 
     /** {@inheritDoc} */
@@ -459,6 +459,18 @@ public class IgniteProcessProxy implements IgniteEx {
         return proc;
     }
 
+    /**
+     * @return {@link IgniteCompute} instance to communicate with remote node.
+     */
+    public IgniteCompute remoteCompute() {
+        ClusterGroup grp = localJvmGrid().cluster().forNodeId(id);
+
+        if (grp.nodes().isEmpty())
+            throw new IllegalStateException("Could not found node with id=" + id + ".");
+
+        return locJvmGrid.compute(grp);
+    }
+
     // TODO delete or use.
 //    public <K, V> GridCacheAdapter<K, V> remoteInternalCache() {
 //        return (GridCacheAdapter<K, V>)compute.call(new MyCallable(id));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/470e48b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
deleted file mode 100644
index 6464838..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
+++ /dev/null
@@ -1,78 +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.multijvm.framework;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.transactions.*;
-
-import java.util.*;
-
-/**
- * Ignite transactions proxy for ignite instance at another JVM.
- */
-public class IgniteTransactionsProcessProxy implements IgniteTransactions {
-    /** Compute. */
-    private final transient IgniteCompute compute;
-
-    /** Grid id. */
-    private final UUID gridId;
-
-    /**
-     * @param proxy Ignite process proxy.
-     */
-    public IgniteTransactionsProcessProxy(IgniteProcessProxy proxy) {
-        gridId = proxy.getId();
-
-        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
-
-        compute = proxy.localJvmGrid().compute(grp);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Transaction txStart() throws IllegalStateException {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Transaction txStart(TransactionConcurrency concurrency, TransactionIsolation isolation) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Transaction txStart(TransactionConcurrency concurrency, TransactionIsolation isolation, long timeout,
-        int txSize) {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Transaction tx() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public TransactionMetrics metrics() {
-        return null; // TODO: CODE: implement.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resetMetrics() {
-        // TODO: CODE: implement.
-    }
-}


[5/6] incubator-ignite git commit: # ignite-648: empty implementation of transactions

Posted by sb...@apache.org.
# ignite-648: empty implementation of transactions


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

Branch: refs/heads/ignite-648
Commit: 2e6fd9b0386bad0836485ef74a18d3dc27837398
Parents: dc674a1
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 17:37:08 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 17:37:08 2015 +0300

----------------------------------------------------------------------
 .../multijvm/framework/IgniteProcessProxy.java  |  2 +-
 .../IgniteTransactionsProcessProxy.java         | 78 ++++++++++++++++++++
 2 files changed, 79 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e6fd9b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
index 95ce583..1d11dc3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
@@ -375,7 +375,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
-        return null; // TODO: CODE: implement.
+        return new IgniteTransactionsProcessProxy(this);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e6fd9b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
new file mode 100644
index 0000000..6464838
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteTransactionsProcessProxy.java
@@ -0,0 +1,78 @@
+/*
+ * 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.multijvm.framework;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.*;
+
+/**
+ * Ignite transactions proxy for ignite instance at another JVM.
+ */
+public class IgniteTransactionsProcessProxy implements IgniteTransactions {
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** Grid id. */
+    private final UUID gridId;
+
+    /**
+     * @param proxy Ignite process proxy.
+     */
+    public IgniteTransactionsProcessProxy(IgniteProcessProxy proxy) {
+        gridId = proxy.getId();
+
+        ClusterGroup grp = proxy.localJvmGrid().cluster().forNodeId(proxy.getId());
+
+        compute = proxy.localJvmGrid().compute(grp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Transaction txStart() throws IllegalStateException {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Transaction txStart(TransactionConcurrency concurrency, TransactionIsolation isolation) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Transaction txStart(TransactionConcurrency concurrency, TransactionIsolation isolation, long timeout,
+        int txSize) {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Transaction tx() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public TransactionMetrics metrics() {
+        return null; // TODO: CODE: implement.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resetMetrics() {
+        // TODO: CODE: implement.
+    }
+}


[4/6] incubator-ignite git commit: # ignite-648: iterator

Posted by sb...@apache.org.
# ignite-648: iterator


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

Branch: refs/heads/ignite-648
Commit: dc674a10e87108206f0ec46dd7f6f8445168d383
Parents: 9cdd165
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 17:23:23 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 17:23:23 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java      |  2 +-
 .../multijvm/framework/IgniteCacheProcessProxy.java  | 15 +++++++++++++--
 2 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc674a10/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 292773c..75cba54 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -164,7 +164,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
                 for (Map.Entry<String, CacheConfiguration[]> entry : cacheCfgMap.entrySet()) {
                     Ignite ignite;
-                    
+
                     ignite = IgnitionEx.grid(entry.getKey());
 
                     for (CacheConfiguration cfg : entry.getValue())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc674a10/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
index 6e09286..61cbdc3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
@@ -501,7 +501,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
     /** {@inheritDoc} */
     @Override public <T> T unwrap(final Class<T> clazz) {
-        throw new UnsupportedOperationException("Method cannot be supported because T can be unmarshalliable.");
+        throw new UnsupportedOperationException("Method cannot be supported because T can be unmarshallable.");
     }
 
     /** {@inheritDoc} */
@@ -516,7 +516,18 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
     /** {@inheritDoc} */
     @Override public Iterator<Entry<K, V>> iterator() {
-        throw new UnsupportedOperationException("Method should be supported.");
+        final Collection<Entry<K, V>> col = (Collection<Entry<K, V>>)compute.call(new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                Collection res = new ArrayList();
+
+                for (Object o : cache())
+                    res.add(o);
+
+                return res;
+            }
+        });
+
+        return col.iterator();
     }
 
     /** {@inheritDoc} */