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/07/01 14:19:15 UTC

incubator-ignite git commit: # ignite-648: IgniteClusterProcessProxy.java instead of local cluster

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 aaca0e8b3 -> ae4e791fd


# ignite-648: IgniteClusterProcessProxy.java instead of local cluster


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

Branch: refs/heads/ignite-648
Commit: ae4e791fd5d31d1cad229510dd3337860eaf2959
Parents: aaca0e8
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 15:20:10 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 15:20:10 2015 +0300

----------------------------------------------------------------------
 .../multijvm/IgniteClusterProcessProxy.java     | 320 +++++++++++++++++++
 .../multijvm/IgniteEventsProcessProxy.java      |  32 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  94 +++---
 3 files changed, 383 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
new file mode 100644
index 0000000..99ec424
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -0,0 +1,320 @@
+/*
+ * 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.testframework.junits.multijvm;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Proxy class for cluster at another jvm.
+ */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
+public class IgniteClusterProcessProxy implements IgniteClusterEx {
+    /** Grid id. */
+    private final UUID gridId;
+
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** */
+    private final IgniteProcessProxy proxy;
+
+    /**
+     * @param proxy Ignite Proxy.
+     */
+    public IgniteClusterProcessProxy(IgniteProcessProxy proxy) {
+        this.proxy = proxy;
+        gridId = proxy.getId();
+        compute = proxy.remoteCompute();
+    }
+
+    /**
+     * Returns cluster instance. Method to be called from closure at another JVM.
+     *
+     * @return Cache.
+     */
+    private IgniteClusterEx cluster() {
+        return (IgniteClusterEx)Ignition.ignite(gridId).cluster();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroupEx forSubjectId(final UUID subjId) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes,
+        boolean clientNodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode localNode() {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().localNode();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forLocal() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> ConcurrentMap<K, V> nodeLocalMap() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public long topologyVersion() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> topology(long topVer) throws UnsupportedOperationException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(@Nullable String cacheName,
+        @Nullable Collection<? extends K> keys) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, K key) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterStartNodeResult> startNodes(File file, boolean restart, int timeout,
+        int maxConn) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterStartNodeResult> startNodes(Collection<Map<String, Object>> hosts,
+        @Nullable Map<String, Object> dflts, boolean restart, int timeout, int maxConn) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopNodes() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopNodes(Collection<UUID> ids) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restartNodes() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restartNodes(Collection<UUID> ids) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resetMetrics() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCluster withAsync() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Ignite ignite() {
+        return proxy;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodes(Collection<? extends ClusterNode> nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNode(ClusterNode node, ClusterNode... nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOthers(ClusterNode node, ClusterNode... nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOthers(ClusterGroup prj) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodeIds(Collection<UUID> ids) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodeId(UUID id, UUID... ids) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forPredicate(IgnitePredicate<ClusterNode> p) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forAttribute(String name, @Nullable Object val) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forServers() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forClients() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forCacheNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forDataNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forClientNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forRemotes() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forHost(ClusterNode node) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forHost(String host, String... hosts) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forDaemons() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forRandom() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOldest() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forYoungest() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> nodes() {
+        return compute.call(new IgniteCallable<Collection<ClusterNode>>() {
+            @Override public Collection<ClusterNode> call() throws Exception {
+                return cluster().nodes();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode node(final UUID nid) {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().node(nid);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode node() {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().node();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> hostNames() {
+        return compute.call(new IgniteCallable<Collection<String>>() {
+            @Override public Collection<String> call() throws Exception {
+                return cluster().hostNames();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgnitePredicate<ClusterNode> predicate() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterMetrics metrics() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
index eb09965..018aa8d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
@@ -54,47 +54,47 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public ClusterGroup clusterGroup() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
         @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> UUID remoteListen(@Nullable IgniteBiPredicate<UUID, T> locLsnr,
         @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@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 {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void stopRemoteListen(UUID opId) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
         @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void recordLocal(Event evt) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -108,41 +108,41 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void enableLocal(int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void disableLocal(int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public int[] enabledEvents() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isEnabled(int type) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteEvents withAsync() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isAsync() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <R> IgniteFuture<R> future() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 5ed5b87..b96fe5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -77,7 +77,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
         List<String> jvmArgs = U.jvmArgs();
 
-        List<String> filteredJvmArgs = new ArrayList<>();
+        Collection<String> filteredJvmArgs = new ArrayList<>();
 
         for (String arg : jvmArgs) {
             if(!arg.toLowerCase().startsWith("-agentlib"))
@@ -195,63 +195,63 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex(@Nullable String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteInternalCache<?, ?>> cachesx(
         @Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean eventUserRecordable(int type) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean allEventsUserRecordable(int[] types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isJmxRemoteEnabled() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isRestartEnabled() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Hadoop hadoop() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteClusterEx cluster() {
-        return (IgniteClusterEx)locJvmGrid.cluster();
+        return new IgniteClusterProcessProxy(this);
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public String latestVersion() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -265,27 +265,27 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public GridKernalContext context() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -295,85 +295,85 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteEvents events(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteProductVersion version() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteScheduler scheduler() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -383,7 +383,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
-        return new IgniteCacheProcessProxy(name, this);
+        return new IgniteCacheProcessProxy<>(name, this);
     }
 
     /** {@inheritDoc} */
@@ -393,61 +393,61 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFileSystem fileSystem(String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFileSystem> fileSystems() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteAtomicReference<T> atomicReference(String name, @Nullable T initVal,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <T, S> IgniteAtomicStamped<T, S> atomicStamped(String name, @Nullable T initVal, @Nullable S initStamp,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteQueue<T> queue(String name, int cap,
         @Nullable CollectionConfiguration cfg) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteSet<T> set(String name, @Nullable CollectionConfiguration cfg) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -462,7 +462,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K> Affinity<K> affinity(String cacheName) {
-        return new AffinityProcessProxy(cacheName, this);
+        return new AffinityProcessProxy<>(cacheName, this);
     }
 
     /**