You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/08/31 16:10:54 UTC

[01/18] ignite git commit: ignite-1124 Changed synchronization in GridNearAtomicUpdateFuture to avoid races, added tests for retries

Repository: ignite
Updated Branches:
  refs/heads/ignite-843 09d3a7bf9 -> 1f1659492


http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 0ab5729..e113fcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -32,6 +33,7 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.transactions.TransactionConcurrency.*;
 import static org.apache.ignite.transactions.TransactionIsolation.*;
 
@@ -44,12 +46,12 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
 
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.TRANSACTIONAL;
+        return TRANSACTIONAL;
     }
 
     /** {@inheritDoc} */
-    @Override protected int keysCount() {
-        return 20_000;
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
     }
 
     /**
@@ -74,7 +76,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
             }
         });
 
-        int keysCnt = keysCount();
+        final int keysCnt = 20_000;
 
         try {
             for (int i = 0; i < keysCnt; i++)
@@ -90,6 +92,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     public void testExplicitTransactionRetries() throws Exception {
         final AtomicInteger idx = new AtomicInteger();
         int threads = 8;
@@ -97,8 +100,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         final AtomicReferenceArray<Exception> err = new AtomicReferenceArray<>(threads);
 
         IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 int th = idx.getAndIncrement();
                 int base = th * FACTOR;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
index 68c7fbb..f556023 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
@@ -26,11 +26,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  */
 public class IgniteCacheAtomicReplicatedNodeRestartSelfTest extends GridCacheReplicatedNodeRestartSelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1124");
-    }
-
-    /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java
index 5b9af4f..5bb1706 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java
@@ -22,6 +22,8 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.testframework.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
 /**
  *
  */
@@ -37,11 +39,6 @@ public class IgniteCacheSslStartStopSelfTest extends IgniteCachePutRetryAbstract
 
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int keysCount() {
-        return 60_000;
+        return ATOMIC;
     }
 }


[03/18] ignite git commit: Platforms: minor refactoring.

Posted by ak...@apache.org.
Platforms: minor refactoring.


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

Branch: refs/heads/ignite-843
Commit: e6de574afd23bf2442ce816d181a7484ddbb6fcd
Parents: 8b625a3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 09:38:16 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 09:38:16 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformTarget.java     | 22 ++++++++++++++++++++
 .../platform/PlatformAbstractTarget.java        | 21 ++++---------------
 2 files changed, 26 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e6de574a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
index 1d54b4e..b444247 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform;
 
+import org.apache.ignite.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -73,4 +74,25 @@ public interface PlatformTarget {
      * @throws Exception In case of failure.
      */
     public void inOutOp(int type, long inMemPtr, long outMemPtr, @Nullable Object arg) throws Exception;
+
+    /**
+     * Start listening for the future.
+     *
+     * @param futId Future ID.
+     * @param typ Result type.
+     * @throws IgniteCheckedException In case of failure.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void listenFuture(final long futId, int typ) throws Exception;
+
+    /**
+     * Start listening for the future for specific operation type.
+     *
+     * @param futId Future ID.
+     * @param typ Result type.
+     * @param opId Operation ID required to pick correct result writer.
+     * @throws IgniteCheckedException In case of failure.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6de574a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index b68b16e..903df0a 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -142,26 +142,13 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
         return platformCtx;
     }
 
-    /**
-     * Start listening for the future.
-     *
-     * @param futId Future ID.
-     * @param typ Result type.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public void listenFuture(final long futId, int typ) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public void listenFuture(final long futId, int typ) throws Exception {
         PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, null);
     }
 
-    /**
-     * Start listening for the future.
-     *
-     * @param futId Future ID.
-     * @param typ Result type.
-     * @param opId Operation ID required to pick correct result writer.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public void listenFuture(final long futId, int typ, int opId) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
         PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, futureWriter(opId));
     }
 


[02/18] ignite git commit: ignite-1124 Changed synchronization in GridNearAtomicUpdateFuture to avoid races, added tests for retries

Posted by ak...@apache.org.
ignite-1124 Changed synchronization in GridNearAtomicUpdateFuture to avoid races, added tests for retries


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

Branch: refs/heads/ignite-843
Commit: 8b625a3a9ac602f8374b74a46ff70c88ef1c9014
Parents: 9f7dc50
Author: sboikov <sb...@gridgain.com>
Authored: Mon Aug 31 09:14:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Aug 31 09:14:28 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAtomicFuture.java |    5 -
 .../processors/cache/GridCacheMvccManager.java  |   62 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    6 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    7 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 1235 +++++++++---------
 .../util/future/GridCompoundFuture.java         |    3 +-
 .../apache/ignite/internal/util/typedef/X.java  |    1 +
 ...eCacheEntryListenerEagerTtlDisabledTest.java |    4 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  282 +++-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |   77 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   14 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |    5 -
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |    9 +-
 13 files changed, 948 insertions(+), 762 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
index 8724d3a..e64a9e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicFuture.java
@@ -27,11 +27,6 @@ import java.util.*;
  */
 public interface GridCacheAtomicFuture<R> extends GridCacheFuture<R> {
     /**
-     * @return Future topology version.
-     */
-    public AffinityTopologyVersion topologyVersion();
-
-    /**
      * Gets future that will be completed when it is safe when update is finished on the given version of topology.
      *
      * @param topVer Topology version to finish.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 6a8c6fe..bbac42b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -196,8 +196,12 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
 
                     cacheFut.onNodeLeft(discoEvt.eventNode().id());
 
-                    if (cacheFut.isCancelled() || cacheFut.isDone())
-                        atomicFuts.remove(cacheFut.futureId(), fut);
+                    if (cacheFut.isCancelled() || cacheFut.isDone()) {
+                        GridCacheVersion futVer = cacheFut.version();
+
+                        if (futVer != null)
+                            atomicFuts.remove(futVer, fut);
+                    }
                 }
             }
         }
@@ -347,16 +351,6 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * @param fut Future to check.
-     * @return {@code True} if future is registered.
-     */
-    public boolean hasFuture(GridCacheFuture<?> fut) {
-        assert fut != null;
-
-        return future(fut.version(), fut.futureId()) != null;
-    }
-
-    /**
      * @param futVer Future ID.
      * @param fut Future.
      */
@@ -565,6 +559,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @param ver Version.
      * @return All futures for given lock version.
      */
+    @SuppressWarnings("unchecked")
     public <T> Collection<? extends IgniteInternalFuture<T>> futures(GridCacheVersion ver) {
         Collection c = futs.get(ver);
 
@@ -572,6 +567,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param cacheCtx Cache context.
      * @param ver Lock version to check.
      * @return {@code True} if lock had been removed.
      */
@@ -580,6 +576,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param cacheCtx Cache context.
      * @param ver Obsolete entry version.
      * @return {@code True} if added.
      */
@@ -688,27 +685,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * @param keys Keys.
-     * @param base Base version.
-     * @return Versions that are less than {@code base} whose keys are in the {@code keys} collection.
-     */
-    public Collection<GridCacheVersion> localDhtPendingVersions(Collection<KeyCacheObject> keys, GridCacheVersion base) {
-        Collection<GridCacheVersion> lessPending = new GridLeanSet<>(5);
-
-        for (GridCacheMvccCandidate cand : dhtLocCands) {
-            if (cand.version().isLess(base)) {
-                if (keys.contains(cand.key()))
-                    lessPending.add(cand.version());
-            }
-            else
-                break;
-        }
-
-        return lessPending;
-    }
-
-    /**
-     *
+     * @param cacheCtx Cache context.
      * @param cand Cache lock candidate to add.
      * @return {@code True} if added as a result of this operation,
      *      {@code false} if was previously added.
@@ -924,24 +901,6 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         X.println(">>>   finishFutsSize: " + finishFuts.size());
     }
 
-
-    /**
-     * @param nodeId Node ID.
-     * @return Filter.
-     */
-    private IgnitePredicate<GridCacheMvccCandidate> nodeIdFilter(final UUID nodeId) {
-        if (nodeId == null)
-            return F.alwaysTrue();
-
-        return new P1<GridCacheMvccCandidate>() {
-            @Override public boolean apply(GridCacheMvccCandidate c) {
-                UUID otherId = c.otherNodeId();
-
-                return c.nodeId().equals(nodeId) || (otherId != null && otherId.equals(nodeId));
-            }
-        };
-    }
-
     /**
      * @param topVer Topology version.
      * @return Future that signals when all locks for given partitions are released.
@@ -994,6 +953,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      *
      * @return Finish update future.
      */
+    @SuppressWarnings("unchecked")
     public IgniteInternalFuture<?> finishAtomicUpdates(AffinityTopologyVersion topVer) {
         GridCompoundFuture<Object, Object> res = new GridCompoundFuture<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 4b8585e..0985ae3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -135,6 +135,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
             @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
                 if (ctx.config().getAtomicWriteOrderMode() == CLOCK) {
+                    assert req.writeSynchronizationMode() != FULL_ASYNC : req;
+
                     // Always send reply in CLOCK ordering mode.
                     sendNearUpdateReply(res.nodeId(), res);
 
@@ -2247,6 +2249,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param req Request to remap.
      */
     private void remapToNewPrimary(GridNearAtomicUpdateRequest req) {
+        assert req.writeSynchronizationMode() == FULL_ASYNC : req;
+
         if (log.isDebugEnabled())
             log.debug("Remapping near update request locally: " + req);
 
@@ -2279,7 +2283,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             drRmvVals = null;
         }
         else {
-            assert req.operation() == DELETE;
+            assert req.operation() == DELETE : req;
 
             drRmvVals = req.conflictVersions();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 4b1a58f..04128b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -168,13 +168,8 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
     }
 
     /** {@inheritDoc} */
-    @Override public AffinityTopologyVersion topologyVersion() {
-        return updateReq.topologyVersion();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
-        if (waitForExchange && topologyVersion().compareTo(topVer) < 0)
+        if (waitForExchange && updateReq.topologyVersion().compareTo(topVer) < 0)
             return this;
 
         return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 07ec808..3f22808 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.affinity.*;
@@ -36,11 +35,9 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
-import org.jsr166.*;
 
 import javax.cache.expiry.*;
 import java.util.*;
-import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
@@ -64,9 +61,6 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /** Cache. */
     private GridDhtAtomicCache cache;
 
-    /** Future ID. */
-    private volatile GridCacheVersion futVer;
-
     /** Update operation. */
     private final GridCacheOperation op;
 
@@ -88,55 +82,24 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
     private Collection<GridCacheVersion> conflictRmvVals;
 
-    /** Mappings. */
-    @GridToStringInclude
-    private ConcurrentMap<UUID, GridNearAtomicUpdateRequest> mappings;
-
-    /** Error. */
-    private volatile CachePartialUpdateCheckedException err;
-
-    /** Operation result. */
-    private volatile GridCacheReturn opRes;
-
     /** Return value require flag. */
     private final boolean retval;
 
     /** Expiry policy. */
     private final ExpiryPolicy expiryPlc;
 
-    /** Future map topology version. */
-    private volatile AffinityTopologyVersion topVer = AffinityTopologyVersion.ZERO;
-
-    /** Completion future for a particular topology version. */
-    private GridFutureAdapter<Void> topCompleteFut;
-
     /** Optional filter. */
     private final CacheEntryPredicate[] filter;
 
     /** Write synchronization mode. */
     private final CacheWriteSynchronizationMode syncMode;
 
-    /** If this future mapped to single node. */
-    private volatile Boolean single;
-
-    /** If this future is mapped to a single node, this field will contain that node ID. */
-    private UUID singleNodeId;
-
-    /** Single update request. */
-    private GridNearAtomicUpdateRequest singleReq;
-
     /** Raw return value flag. */
     private final boolean rawRetval;
 
     /** Fast map flag. */
     private final boolean fastMap;
 
-    /** */
-    private boolean fastMapRemap;
-
-    /** */
-    private GridCacheVersion updVer;
-
     /** Near cache flag. */
     private final boolean nearEnabled;
 
@@ -156,7 +119,10 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     private final boolean waitTopFut;
 
     /** Remap count. */
-    private AtomicInteger remapCnt;
+    private int remapCnt;
+
+    /** State. */
+    private final UpdateState state;
 
     /**
      * @param cctx Cache context.
@@ -175,6 +141,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip store flag.
+     * @param remapCnt Maximum number of retries.
+     * @param waitTopFut If {@code false} does not wait for affinity change future.
      */
     public GridNearAtomicUpdateFuture(
         GridCacheContext cctx,
@@ -223,8 +191,6 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         if (log == null)
             log = U.logger(cctx.kernalContext(), logRef, GridFutureAdapter.class);
 
-        mappings = new ConcurrentHashMap8<>(keys.size(), 1.0f);
-
         fastMap = F.isEmpty(filter) && op != TRANSFORM && cctx.config().getWriteSynchronizationMode() == FULL_SYNC &&
             cctx.config().getAtomicWriteOrderMode() == CLOCK &&
             !(cctx.writeThrough() && cctx.config().getInterceptor() != null);
@@ -234,22 +200,24 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         if (!waitTopFut)
             remapCnt = 1;
 
-        this.remapCnt = new AtomicInteger(remapCnt);
+        this.remapCnt = remapCnt;
+
+        state = new UpdateState();
     }
 
     /** {@inheritDoc} */
     @Override public IgniteUuid futureId() {
-        return futVer.asGridUuid();
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public GridCacheVersion version() {
-        return futVer;
+        return state.futureVersion();
     }
 
     /** {@inheritDoc} */
     @Override public Collection<? extends ClusterNode> nodes() {
-        return F.view(F.viewReadOnly(mappings.keySet(), U.id2Node(cctx.kernalContext())), F.notNull());
+        throw new UnsupportedOperationException();
     }
 
     /**
@@ -261,45 +229,13 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     }
 
     /** {@inheritDoc} */
-    @Override public AffinityTopologyVersion topologyVersion() {
-        return topVer;
-    }
-
-    /** {@inheritDoc} */
     @Override public Collection<?> keys() {
         return keys;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        Boolean single0 = single;
-
-        if (single0 != null && single0) {
-            if (singleNodeId.equals(nodeId)) {
-                onDone(addFailedKeys(
-                    singleReq.keys(),
-                    singleReq.topologyVersion(),
-                    new ClusterTopologyCheckedException("Primary node left grid before response is received: " + nodeId)));
-
-                return true;
-            }
-
-            return false;
-        }
-
-        GridNearAtomicUpdateRequest req = mappings.get(nodeId);
-
-        if (req != null) {
-            addFailedKeys(req.keys(),
-                req.topologyVersion(),
-                new ClusterTopologyCheckedException("Primary node left grid before response is received: " + nodeId));
-
-            mappings.remove(nodeId);
-
-            checkComplete();
-
-            return true;
-        }
+        state.onNodeLeft(nodeId);
 
         return false;
     }
@@ -329,142 +265,34 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             topVer = cctx.mvcc().lastExplicitLockTopologyVersion(Thread.currentThread().getId());
 
         if (topVer == null)
-            mapOnTopology(null, false, null);
+            mapOnTopology();
         else {
             topLocked = true;
 
             // Cannot remap.
-            remapCnt.set(1);
+            remapCnt = 1;
 
-            map0(topVer, null, false, null);
+            state.map(topVer);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
-        if (waitForPartitionExchange() && topologyVersion().compareTo(topVer) < 0) {
-            GridFutureAdapter<Void> fut = null;
-
-            synchronized (this) {
-                if (this.topVer == AffinityTopologyVersion.ZERO)
-                    return null;
+        if (waitForPartitionExchange()) {
+            GridFutureAdapter<Void> fut = state.completeFuture(topVer);
 
-                if (this.topVer.compareTo(topVer) < 0) {
-                    if (topCompleteFut == null)
-                        topCompleteFut = new GridFutureAdapter<>();
+            if (fut != null && isDone()) {
+                fut.onDone();
 
-                    fut = topCompleteFut;
-                }
+                return null;
             }
 
-            if (fut != null && isDone())
-                fut.onDone();
-
             return fut;
         }
 
         return null;
     }
 
-    /**
-     * @param failed Keys to remap.
-     * @param errTopVer Topology version for failed update.
-     */
-    private void remap(Collection<?> failed, AffinityTopologyVersion errTopVer) {
-        assert errTopVer != null;
-
-        GridCacheVersion futVer0 = futVer;
-
-        if (futVer0 == null || cctx.mvcc().removeAtomicFuture(futVer0) == null)
-            return;
-
-        Collection<Object> remapKeys = new ArrayList<>(failed.size());
-        Collection<Object> remapVals = vals != null ? new ArrayList<>(failed.size()) : null;
-        Collection<GridCacheDrInfo> remapConflictPutVals = conflictPutVals != null ? new ArrayList<GridCacheDrInfo>(failed.size()) : null;
-        Collection<GridCacheVersion> remapConflictRmvVals = conflictRmvVals != null ? new ArrayList<GridCacheVersion>(failed.size()) : null;
-
-        Iterator<?> keyIt = keys.iterator();
-        Iterator<?> valsIt = vals != null ? vals.iterator() : null;
-        Iterator<GridCacheDrInfo> conflictPutValsIt = conflictPutVals != null ? conflictPutVals.iterator() : null;
-        Iterator<GridCacheVersion> conflictRmvValsIt = conflictRmvVals != null ? conflictRmvVals.iterator() : null;
-
-        for (Object key : failed) {
-            while (keyIt.hasNext()) {
-                Object nextKey = keyIt.next();
-                Object nextVal = valsIt != null ? valsIt.next() : null;
-                GridCacheDrInfo nextConflictPutVal = conflictPutValsIt != null ? conflictPutValsIt.next() : null;
-                GridCacheVersion nextConflictRmvVal = conflictRmvValsIt != null ? conflictRmvValsIt.next() : null;
-
-                if (F.eq(key, nextKey)) {
-                    remapKeys.add(nextKey);
-
-                    if (remapVals != null)
-                        remapVals.add(nextVal);
-
-                    if (remapConflictPutVals != null)
-                        remapConflictPutVals.add(nextConflictPutVal);
-
-                    if (remapConflictRmvVals != null)
-                        remapConflictRmvVals.add(nextConflictRmvVal);
-
-                    break;
-                }
-            }
-        }
-
-        keys = remapKeys;
-        vals = remapVals;
-        conflictPutVals = remapConflictPutVals;
-        conflictRmvVals = remapConflictRmvVals;
-
-        single = null;
-        futVer = null;
-        err = null;
-        opRes = null;
-
-        GridFutureAdapter<Void> fut0;
-
-        synchronized (this) {
-            mappings = new ConcurrentHashMap8<>(keys.size(), 1.0f);
-
-            assert topVer != null && topVer.topologyVersion() > 0 : this;
-
-            topVer = AffinityTopologyVersion.ZERO;
-
-            fut0 = topCompleteFut;
-
-            topCompleteFut = null;
-        }
-
-        if (fut0 != null)
-            fut0.onDone();
-
-        singleNodeId = null;
-        singleReq = null;
-        fastMapRemap = false;
-        updVer = null;
-        topLocked = false;
-
-        IgniteInternalFuture<?> fut = cctx.affinity().affinityReadyFuture(errTopVer.topologyVersion() + 1);
-
-        fut.listen(new CI1<IgniteInternalFuture<?>>() {
-            @Override public void apply(final IgniteInternalFuture<?> fut) {
-                cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            fut.get();
-
-                            map();
-                        }
-                        catch (IgniteCheckedException e) {
-                            onDone(e);
-                        }
-                    }
-                });
-            }
-        });
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings("ConstantConditions")
     @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
@@ -478,35 +306,11 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         if (op == TRANSFORM && retval == null)
             retval = Collections.emptyMap();
 
-        if (err != null && X.hasCause(err, CachePartialUpdateCheckedException.class) &&
-            X.hasCause(err, ClusterTopologyCheckedException.class) &&
-            storeFuture() &&
-            remapCnt.decrementAndGet() > 0) {
-            ClusterTopologyCheckedException topErr = X.cause(err, ClusterTopologyCheckedException.class);
-
-            if (!(topErr instanceof  ClusterTopologyServerNotFoundException)) {
-                CachePartialUpdateCheckedException cause = X.cause(err, CachePartialUpdateCheckedException.class);
-
-                assert cause != null && cause.topologyVersion() != null : err;
-
-                remap(cause.failedKeys(), cause.topologyVersion());
-
-                return false;
-            }
-        }
-
         if (super.onDone(retval, err)) {
-            if (futVer != null)
-                cctx.mvcc().removeAtomicFuture(version());
-
-            GridFutureAdapter<Void> fut0;
-
-            synchronized (this) {
-                fut0 = topCompleteFut;
-            }
+            GridCacheVersion futVer = state.onFutureDone();
 
-            if (fut0 != null)
-                fut0.onDone();
+            if (futVer != null)
+                cctx.mvcc().removeAtomicFuture(futVer);
 
             return true;
         }
@@ -521,68 +325,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
      * @param res Update response.
      */
     public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res) {
-        if (res.remapKeys() != null) {
-            assert !fastMap || cctx.kernalContext().clientNode();
-
-            Collection<KeyCacheObject> remapKeys = fastMap ? null : res.remapKeys();
-
-            mapOnTopology(remapKeys, true, nodeId);
-
-            return;
-        }
-
-        GridCacheReturn ret = res.returnValue();
-
-        Boolean single0 = single;
-
-        if (single0 != null && single0) {
-            assert singleNodeId.equals(nodeId) : "Invalid response received for single-node mapped future " +
-                "[singleNodeId=" + singleNodeId + ", nodeId=" + nodeId + ", res=" + res + ']';
-
-            updateNear(singleReq, res);
-
-            if (res.error() != null) {
-                onDone(res.failedKeys() != null ?
-                    addFailedKeys(res.failedKeys(), singleReq.topologyVersion(), res.error()) : res.error());
-            }
-            else {
-                if (op == TRANSFORM) {
-                    if (ret != null)
-                        addInvokeResults(ret);
-
-                    onDone(opRes);
-                }
-                else {
-                    GridCacheReturn opRes0 = opRes = ret;
-
-                    onDone(opRes0);
-                }
-            }
-        }
-        else {
-            GridNearAtomicUpdateRequest req = mappings.get(nodeId);
-
-            if (req != null) { // req can be null if onResult is being processed concurrently with onNodeLeft.
-                updateNear(req, res);
-
-                if (res.error() != null)
-                    addFailedKeys(req.keys(), req.topologyVersion(), res.error());
-                else {
-                    if (op == TRANSFORM) {
-                        assert !req.fastMap();
-
-                        if (ret != null)
-                            addInvokeResults(ret);
-                    }
-                    else if (req.fastMap() && req.hasPrimary())
-                        opRes = ret;
-                }
-
-                mappings.remove(nodeId);
-            }
-
-            checkComplete();
-        }
+        state.onResult(nodeId, res, false);
     }
 
     /**
@@ -602,12 +345,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
     /**
      * Maps future on ready topology.
-     *
-     * @param keys Keys to map.
-     * @param remap Boolean flag indicating if this is partial future remap.
-     * @param oldNodeId Old node ID if remap.
      */
-    private void mapOnTopology(final Collection<?> keys, final boolean remap, final UUID oldNodeId) {
+    private void mapOnTopology() {
         cache.topology().readLock();
 
         AffinityTopologyVersion topVer = null;
@@ -634,11 +373,13 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             }
             else {
                 if (waitTopFut) {
+                    assert !topLocked : this;
+
                     fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
                             cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                                 @Override public void run() {
-                                    mapOnTopology(keys, remap, oldNodeId);
+                                    mapOnTopology();
                                 }
                             });
                         }
@@ -654,232 +395,543 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             cache.topology().readUnlock();
         }
 
-        map0(topVer, keys, remap, oldNodeId);
+        state.map(topVer);
     }
 
     /**
-     * Checks if future is ready to be completed.
+     * @return {@code True} future is stored by {@link GridCacheMvccManager#addAtomicFuture}.
      */
-    private void checkComplete() {
-        boolean remap = false;
+    private boolean storeFuture() {
+        return cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC;
+    }
 
-        synchronized (this) {
-            if (topVer != AffinityTopologyVersion.ZERO &&
-                ((syncMode == FULL_ASYNC && cctx.config().getAtomicWriteOrderMode() == PRIMARY) || mappings.isEmpty())) {
-                CachePartialUpdateCheckedException err0 = err;
+    /**
+     * Maps key to nodes. If filters are absent and operation is not TRANSFORM, then we can assign version on near
+     * node and send updates in parallel to all participating nodes.
+     *
+     * @param key Key to map.
+     * @param topVer Topology version to map.
+     * @param fastMap Flag indicating whether mapping is performed for fast-circuit update.
+     * @return Collection of nodes to which key is mapped.
+     */
+    private Collection<ClusterNode> mapKey(
+        KeyCacheObject key,
+        AffinityTopologyVersion topVer,
+        boolean fastMap
+    ) {
+        GridCacheAffinityManager affMgr = cctx.affinity();
 
-                if (err0 != null)
-                    onDone(err0);
-                else {
-                    if (fastMapRemap) {
-                        assert cctx.kernalContext().clientNode();
+        // If we can send updates in parallel - do it.
+        return fastMap ?
+            cctx.topology().nodes(affMgr.partition(key), topVer) :
+            Collections.singletonList(affMgr.primary(key, topVer));
+    }
 
-                        remap = true;
+    /**
+     * Maps future to single node.
+     *
+     * @param nodeId Node ID.
+     * @param req Request.
+     */
+    private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) {
+        if (cctx.localNodeId().equals(nodeId)) {
+            cache.updateAllAsyncInternal(nodeId, req,
+                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
+                    @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
+                        onResult(res.nodeId(), res);
                     }
-                    else
-                        onDone(opRes);
-                }
-            }
+                });
         }
+        else {
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Sending near atomic update request [nodeId=" + req.nodeId() + ", req=" + req + ']');
 
-        if (remap)
-            mapOnTopology(null, true, null);
-    }
+                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
 
-    /**
-     * @return {@code True} future is stored by {@link GridCacheMvccManager#addAtomicFuture}.
-     */
-    private boolean storeFuture() {
-        return cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC;
+                if (syncMode == FULL_ASYNC)
+                    onDone(new GridCacheReturn(cctx, true, null, true));
+            }
+            catch (IgniteCheckedException e) {
+                state.onSendError(req, e);
+            }
+        }
     }
 
     /**
-     * @param topVer Topology version.
-     * @param remapKeys Keys to remap or {@code null} to map all keys.
-     * @param remap Flag indicating if this is partial remap for this future.
-     * @param oldNodeId Old node ID if was remap.
+     * Sends messages to remote nodes and updates local cache.
+     *
+     * @param mappings Mappings to send.
      */
-    private void map0(
-        AffinityTopologyVersion topVer,
-        @Nullable Collection<?> remapKeys,
-        boolean remap,
-        @Nullable UUID oldNodeId) {
-        assert oldNodeId == null || remap || fastMapRemap;
+    private void doUpdate(Map<UUID, GridNearAtomicUpdateRequest> mappings) {
+        UUID locNodeId = cctx.localNodeId();
 
-        Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
+        GridNearAtomicUpdateRequest locUpdate = null;
 
-        if (F.isEmpty(topNodes)) {
-            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
-                "left the grid)."));
+        // Send messages to remote nodes first, then run local update.
+        for (GridNearAtomicUpdateRequest req : mappings.values()) {
+            if (locNodeId.equals(req.nodeId())) {
+                assert locUpdate == null : "Cannot have more than one local mapping [locUpdate=" + locUpdate +
+                    ", req=" + req + ']';
 
-            return;
+                locUpdate = req;
+            }
+            else {
+                try {
+                    if (log.isDebugEnabled())
+                        log.debug("Sending near atomic update request [nodeId=" + req.nodeId() + ", req=" + req + ']');
+
+                    cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
+                }
+                catch (IgniteCheckedException e) {
+                    state.onSendError(req, e);
+                }
+            }
         }
 
-        if (futVer == null)
-            // Assign future version in topology read lock before first exception may be thrown.
-            futVer = cctx.versions().next(topVer);
+        if (locUpdate != null) {
+            cache.updateAllAsyncInternal(cctx.localNodeId(), locUpdate,
+                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
+                    @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
+                        onResult(res.nodeId(), res);
+                    }
+                });
+        }
 
-        if (!remap && storeFuture())
-            cctx.mvcc().addAtomicFuture(version(), this);
+        if (syncMode == FULL_ASYNC)
+            onDone(new GridCacheReturn(cctx, true, null, true));
+    }
 
-        CacheConfiguration ccfg = cctx.config();
+    /**
+     *
+     */
+    private class UpdateState {
+        /** Current topology version. */
+        private AffinityTopologyVersion topVer = AffinityTopologyVersion.ZERO;
 
-        // Assign version on near node in CLOCK ordering mode even if fastMap is false.
-        if (updVer == null)
-            updVer = ccfg.getAtomicWriteOrderMode() == CLOCK ? cctx.versions().next(topVer) : null;
+        /** */
+        private GridCacheVersion updVer;
 
-        if (updVer != null && log.isDebugEnabled())
-            log.debug("Assigned fast-map version for update on near node: " + updVer);
+        /** Topology version when got mapping error. */
+        private AffinityTopologyVersion mapErrTopVer;
 
-        if (keys.size() == 1 && !fastMap && (single == null || single)) {
-            assert remapKeys == null || remapKeys.size() == 1 : remapKeys;
+        /** Mappings if operations is mapped to more than one node. */
+        @GridToStringInclude
+        private Map<UUID, GridNearAtomicUpdateRequest> mappings;
 
-            Object key = F.first(keys);
+        /** Error. */
+        private CachePartialUpdateCheckedException err;
 
-            Object val;
-            GridCacheVersion conflictVer;
-            long conflictTtl;
-            long conflictExpireTime;
+        /** Future ID. */
+        private GridCacheVersion futVer;
 
-            if (vals != null) {
-                // Regular PUT.
-                val = F.first(vals);
-                conflictVer = null;
-                conflictTtl = CU.TTL_NOT_CHANGED;
-                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
-            }
-            else if (conflictPutVals != null) {
-                // Conflict PUT.
-                GridCacheDrInfo conflictPutVal = F.first(conflictPutVals);
+        /** Completion future for a particular topology version. */
+        private GridFutureAdapter<Void> topCompleteFut;
 
-                val = conflictPutVal.value();
-                conflictVer = conflictPutVal.version();
-                conflictTtl = conflictPutVal.ttl();
-                conflictExpireTime = conflictPutVal.expireTime();
-            }
-            else if (conflictRmvVals != null) {
-                // Conflict REMOVE.
-                val = null;
-                conflictVer = F.first(conflictRmvVals);
-                conflictTtl = CU.TTL_NOT_CHANGED;
-                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
-            }
-            else {
-                // Regular REMOVE.
-                val = null;
-                conflictVer = null;
-                conflictTtl = CU.TTL_NOT_CHANGED;
-                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
-            }
+        /** Keys to remap. */
+        private Collection<KeyCacheObject> remapKeys;
 
-            // We still can get here if user pass map with single element.
-            if (key == null) {
-                NullPointerException err = new NullPointerException("Null key.");
+        /** Not null is operation is mapped to single node. */
+        private GridNearAtomicUpdateRequest singleReq;
 
-                onDone(err);
+        /** Operation result. */
+        private GridCacheReturn opRes;
 
-                return;
-            }
+        /**
+         * @return Future version.
+         */
+        @Nullable synchronized GridCacheVersion futureVersion() {
+            return futVer;
+        }
 
-            if (val == null && op != GridCacheOperation.DELETE) {
-                NullPointerException err = new NullPointerException("Null value.");
+        /**
+         * @param nodeId Left node ID.
+         */
+        void onNodeLeft(UUID nodeId) {
+            GridNearAtomicUpdateResponse res = null;
 
-                onDone(err);
+            synchronized (this) {
+                GridNearAtomicUpdateRequest req;
+
+                if (singleReq != null)
+                    req = singleReq.nodeId().equals(nodeId) ? singleReq : null;
+                else
+                    req = mappings != null ? mappings.get(nodeId) : null;
+
+                if (req != null) {
+                    res = new GridNearAtomicUpdateResponse(cctx.cacheId(), nodeId, req.futureVersion());
+
+                    res.addFailedKeys(req.keys(), new ClusterTopologyCheckedException("Primary node left grid before " +
+                        "response is received: " + nodeId));
+                }
+            }
+
+            if (res != null)
+                onResult(nodeId, res, true);
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @param res Response.
+         * @param nodeErr {@code True} if response was created on node failure.
+         */
+        void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
+            GridNearAtomicUpdateRequest req;
+
+            AffinityTopologyVersion remapTopVer = null;
+
+            GridCacheReturn opRes0 = null;
+            CachePartialUpdateCheckedException err0 = null;
+
+            boolean rcvAll;
+
+            GridFutureAdapter<?> fut0 = null;
+
+            synchronized (this) {
+                if (!res.futureVersion().equals(futVer))
+                    return;
+
+                if (singleReq != null) {
+                    if (!singleReq.nodeId().equals(nodeId))
+                        return;
+
+                    req = singleReq;
+
+                    singleReq = null;
+
+                    rcvAll = true;
+                }
+                else {
+                    req = mappings != null ? mappings.remove(nodeId) : null;
+
+                    if (req != null)
+                        rcvAll = mappings.isEmpty();
+                    else
+                        return;
+                }
+
+                assert req != null && req.topologyVersion().equals(topVer) : req;
+
+                if (res.remapKeys() != null) {
+                    assert !fastMap || cctx.kernalContext().clientNode();
+
+                    if (remapKeys == null)
+                        remapKeys = U.newHashSet(res.remapKeys().size());
+
+                    remapKeys.addAll(res.remapKeys());
+
+                    if (mapErrTopVer == null || mapErrTopVer.compareTo(req.topologyVersion()) < 0)
+                        mapErrTopVer = req.topologyVersion();
+                }
+                else if (res.error() != null) {
+                    if (res.failedKeys() != null)
+                        addFailedKeys(res.failedKeys(), req.topologyVersion(), res.error());
+                }
+                else {
+                    if (!req.fastMap() || req.hasPrimary()) {
+                        GridCacheReturn ret = res.returnValue();
+
+                        if (op == TRANSFORM) {
+                            if (ret != null)
+                                addInvokeResults(ret);
+                        }
+                        else
+                            opRes = ret;
+                    }
+                }
+
+                if (rcvAll) {
+                    if (remapKeys != null) {
+                        assert mapErrTopVer != null;
+
+                        remapTopVer = new AffinityTopologyVersion(mapErrTopVer.topologyVersion() + 1);
+                    }
+                    else {
+                        if (err != null &&
+                            X.hasCause(err, CachePartialUpdateCheckedException.class) &&
+                            X.hasCause(err, ClusterTopologyCheckedException.class) &&
+                            storeFuture() &&
+                            --remapCnt > 0) {
+                            ClusterTopologyCheckedException topErr =
+                                X.cause(err, ClusterTopologyCheckedException.class);
+
+                            if (!(topErr instanceof ClusterTopologyServerNotFoundException)) {
+                                CachePartialUpdateCheckedException cause =
+                                    X.cause(err, CachePartialUpdateCheckedException.class);
+
+                                assert cause != null && cause.topologyVersion() != null : err;
+
+                                remapTopVer =
+                                    new AffinityTopologyVersion(cause.topologyVersion().topologyVersion() + 1);
+
+                                err = null;
+
+                                Collection<Object> failedKeys = cause.failedKeys();
+
+                                remapKeys = new ArrayList<>(failedKeys.size());
+
+                                for (Object key : failedKeys)
+                                    remapKeys.add(cctx.toCacheKeyObject(key));
+
+                                updVer = null;
+                            }
+                        }
+                    }
+
+                    if (remapTopVer == null) {
+                        err0 = err;
+                        opRes0 = opRes;
+                    }
+                    else {
+                        fut0 = topCompleteFut;
+
+                        topCompleteFut = null;
+
+                        cctx.mvcc().removeAtomicFuture(futVer);
+
+                        futVer = null;
+                        topVer = AffinityTopologyVersion.ZERO;
+                    }
+                }
+            }
+
+            if (res.error() != null && res.failedKeys() == null) {
+                onDone(res.error());
 
                 return;
             }
 
-            KeyCacheObject cacheKey = cctx.toCacheKeyObject(key);
+            if (!nodeErr && res.remapKeys() == null)
+                updateNear(req, res);
 
-            if (op != TRANSFORM)
-                val = cctx.toCacheObject(val);
+            if (remapTopVer != null) {
+                if (fut0 != null)
+                    fut0.onDone();
 
-            ClusterNode primary = cctx.affinity().primary(cacheKey, topVer);
+                if (!waitTopFut) {
+                    onDone(new GridCacheTryPutFailedException());
+
+                    return;
+                }
+
+                if (topLocked) {
+                    assert !F.isEmpty(remapKeys) : remapKeys;
+
+                    CachePartialUpdateCheckedException e =
+                        new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+
+                    ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException(
+                        "Failed to update keys, topology changed while execute atomic update inside transaction.");
+
+                    cause.retryReadyFuture(cctx.affinity().affinityReadyFuture(remapTopVer));
+
+                    e.add(remapKeys, cause);
+
+                    onDone(e);
+
+                    return;
+                }
+
+                IgniteInternalFuture<AffinityTopologyVersion> fut = cctx.affinity().affinityReadyFuture(remapTopVer);
+
+                fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                        cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                            @Override public void run() {
+                                try {
+                                    AffinityTopologyVersion topVer = fut.get();
+
+                                    map(topVer);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    onDone(e);
+                                }
+                            }
+                        });
+                    }
+                });
 
-            if (primary == null) {
+                return;
+            }
+
+            if (rcvAll)
+                onDone(opRes0, err0);
+        }
+
+        /**
+         * @param req Request.
+         * @param e Error.
+         */
+        void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) {
+            synchronized (this) {
+                GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
+                    req.nodeId(),
+                    req.futureVersion());
+
+                res.addFailedKeys(req.keys(), e);
+
+                onResult(req.nodeId(), res, true);
+            }
+        }
+
+        /**
+         * @param topVer Topology version.
+         */
+        void map(AffinityTopologyVersion topVer) {
+            Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
+
+            if (F.isEmpty(topNodes)) {
                 onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
                     "left the grid)."));
 
                 return;
             }
 
-            GridNearAtomicUpdateRequest req = new GridNearAtomicUpdateRequest(
-                cctx.cacheId(),
-                primary.id(),
-                futVer,
-                fastMap,
-                updVer,
-                topVer,
-                topLocked,
-                syncMode,
-                op,
-                retval,
-                expiryPlc,
-                invokeArgs,
-                filter,
-                subjId,
-                taskNameHash,
-                skipStore,
-                cctx.kernalContext().clientNode());
+            Exception err = null;
+            Map<UUID, GridNearAtomicUpdateRequest> pendingMappings = null;
 
-            req.addUpdateEntry(cacheKey,
-                val,
-                conflictTtl,
-                conflictExpireTime,
-                conflictVer,
-                true);
+            int size = keys.size();
 
             synchronized (this) {
+                assert futVer == null : this;
+                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+
                 this.topVer = topVer;
 
-                single = true;
+                futVer = cctx.versions().next(topVer);
+
+                if (storeFuture())
+                    cctx.mvcc().addAtomicFuture(futVer, GridNearAtomicUpdateFuture.this);
+
+                // Assign version on near node in CLOCK ordering mode even if fastMap is false.
+                if (updVer == null)
+                    updVer = cctx.config().getAtomicWriteOrderMode() == CLOCK ? cctx.versions().next(topVer) : null;
+
+                if (updVer != null && log.isDebugEnabled())
+                    log.debug("Assigned fast-map version for update on near node: " + updVer);
+
+                try {
+                    if (size == 1 && !fastMap) {
+                        assert remapKeys == null || remapKeys.size() == 1;
+
+                        singleReq = mapSingleUpdate();
+                    }
+                    else {
+                        pendingMappings = mapUpdate(topNodes);
+
+                        if (pendingMappings.size() == 1)
+                            singleReq = F.firstValue(pendingMappings);
+                        else {
+                            if (syncMode == PRIMARY_SYNC) {
+                                mappings = U.newHashMap(pendingMappings.size());
+
+                                for (GridNearAtomicUpdateRequest req : pendingMappings.values()) {
+                                    if (req.hasPrimary())
+                                        mappings.put(req.nodeId(), req);
+                                }
+                            }
+                            else
+                                mappings = new HashMap<>(pendingMappings);
+
+                            assert !mappings.isEmpty() || size == 0 : GridNearAtomicUpdateFuture.this;
+                        }
+                    }
+
+                    remapKeys = null;
+                }
+                catch (Exception e) {
+                    err = e;
+                }
+            }
+
+            if (err != null) {
+                onDone(err);
+
+                return;
             }
 
             // Optimize mapping for single key.
-            mapSingle(primary.id(), req);
+            if (singleReq != null)
+                mapSingle(singleReq.nodeId(), singleReq);
+            else {
+                assert pendingMappings != null;
 
-            return;
+                if (size == 0)
+                    onDone(new GridCacheReturn(cctx, true, null, true));
+                else
+                    doUpdate(pendingMappings);
+            }
         }
 
-        Iterator<?> it = null;
+        /**
+         * @param topVer Topology version.
+         * @return Future.
+         */
+        @Nullable synchronized GridFutureAdapter<Void> completeFuture(AffinityTopologyVersion topVer) {
+            if (this.topVer == AffinityTopologyVersion.ZERO)
+                return null;
 
-        if (vals != null)
-            it = vals.iterator();
+            if (this.topVer.compareTo(topVer) < 0) {
+                if (topCompleteFut == null)
+                    topCompleteFut = new GridFutureAdapter<>();
 
-        Iterator<GridCacheDrInfo> conflictPutValsIt = null;
+                return topCompleteFut;
+            }
 
-        if (conflictPutVals != null)
-            conflictPutValsIt = conflictPutVals.iterator();
+            return null;
+        }
 
-        Iterator<GridCacheVersion> conflictRmvValsIt = null;
+        /**
+         * @return Future version.
+         */
+        GridCacheVersion onFutureDone() {
+            GridCacheVersion ver0;
 
-        if (conflictRmvVals != null)
-            conflictRmvValsIt = conflictRmvVals.iterator();
+            GridFutureAdapter<Void> fut0;
 
-        Map<UUID, GridNearAtomicUpdateRequest> pendingMappings = new HashMap<>(topNodes.size(), 1.0f);
+            synchronized (this) {
+                fut0 = topCompleteFut;
 
-        // Must do this in synchronized block because we need to atomically remove and add mapping.
-        // Otherwise checkComplete() may see empty intermediate state.
-        synchronized (this) {
-            if (oldNodeId != null)
-                removeMapping(oldNodeId);
+                topCompleteFut = null;
 
-            // For fastMap mode wait for all responses before remapping.
-            if (remap && fastMap && !mappings.isEmpty()) {
-                fastMapRemap = true;
+                ver0 = futVer;
 
-                return;
+                futVer = null;
             }
 
-            // Create mappings first, then send messages.
-            for (Object key : keys) {
-                if (key == null) {
-                    NullPointerException err = new NullPointerException("Null key.");
+            if (fut0 != null)
+                fut0.onDone();
 
-                    onDone(err);
+            return ver0;
+        }
 
-                    return;
-                }
+        /**
+         * @param topNodes Cache nodes.
+         * @return Mapping.
+         * @throws Exception If failed.
+         */
+        private Map<UUID, GridNearAtomicUpdateRequest> mapUpdate(Collection<ClusterNode> topNodes) throws Exception {
+            Iterator<?> it = null;
+
+            if (vals != null)
+                it = vals.iterator();
+
+            Iterator<GridCacheDrInfo> conflictPutValsIt = null;
+
+            if (conflictPutVals != null)
+                conflictPutValsIt = conflictPutVals.iterator();
+
+            Iterator<GridCacheVersion> conflictRmvValsIt = null;
+
+            if (conflictRmvVals != null)
+                conflictRmvValsIt = conflictRmvVals.iterator();
+
+            Map<UUID, GridNearAtomicUpdateRequest> pendingMappings = U.newHashMap(topNodes.size());
+
+            // Create mappings first, then send messages.
+            for (Object key : keys) {
+                if (key == null)
+                    throw new NullPointerException("Null key.");
 
                 Object val;
                 GridCacheVersion conflictVer;
@@ -892,13 +944,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                     conflictTtl = CU.TTL_NOT_CHANGED;
                     conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
 
-                    if (val == null) {
-                        NullPointerException err = new NullPointerException("Null value.");
-
-                        onDone(err);
-
-                        return;
-                    }
+                    if (val == null)
+                        throw new NullPointerException("Null value.");
                 }
                 else if (conflictPutVals != null) {
                     GridCacheDrInfo conflictPutVal =  conflictPutValsIt.next();
@@ -934,22 +981,16 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
                 Collection<ClusterNode> affNodes = mapKey(cacheKey, topVer, fastMap);
 
-                if (affNodes.isEmpty()) {
-                    onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
-                        "(all partition nodes left the grid)."));
-
-                    return;
-                }
+                if (affNodes.isEmpty())
+                    throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
+                        "(all partition nodes left the grid).");
 
                 int i = 0;
 
                 for (ClusterNode affNode : affNodes) {
-                    if (affNode == null) {
-                        onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
-                            "(all partition nodes left the grid)."));
-
-                        return;
-                    }
+                    if (affNode == null)
+                        throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
+                            "(all partition nodes left the grid).");
 
                     UUID nodeId = affNode.id();
 
@@ -976,11 +1017,6 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                             cctx.kernalContext().clientNode());
 
                         pendingMappings.put(nodeId, mapped);
-
-                        GridNearAtomicUpdateRequest old = mappings.put(nodeId, mapped);
-
-                        assert old == null || (old != null && remap) :
-                            "Invalid mapping state [old=" + old + ", remap=" + remap + ']';
                     }
 
                     mapped.addUpdateEntry(cacheKey, val, conflictTtl, conflictExpireTime, conflictVer, i == 0);
@@ -989,187 +1025,140 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 }
             }
 
-            this.topVer = topVer;
-
-            fastMapRemap = false;
-        }
-
-        if ((single == null || single) && pendingMappings.size() == 1) {
-            Map.Entry<UUID, GridNearAtomicUpdateRequest> entry = F.first(pendingMappings.entrySet());
-
-            single = true;
-
-            mapSingle(entry.getKey(), entry.getValue());
-
-            return;
+            return pendingMappings;
         }
-        else
-            single = false;
-
-        doUpdate(pendingMappings);
-    }
-
-    /**
-     * Maps key to nodes. If filters are absent and operation is not TRANSFORM, then we can assign version on near
-     * node and send updates in parallel to all participating nodes.
-     *
-     * @param key Key to map.
-     * @param topVer Topology version to map.
-     * @param fastMap Flag indicating whether mapping is performed for fast-circuit update.
-     * @return Collection of nodes to which key is mapped.
-     */
-    private Collection<ClusterNode> mapKey(
-        KeyCacheObject key,
-        AffinityTopologyVersion topVer,
-        boolean fastMap
-    ) {
-        GridCacheAffinityManager affMgr = cctx.affinity();
 
-        // If we can send updates in parallel - do it.
-        return fastMap ?
-            cctx.topology().nodes(affMgr.partition(key), topVer) :
-            Collections.singletonList(affMgr.primary(key, topVer));
-    }
-
-    /**
-     * Maps future to single node.
-     *
-     * @param nodeId Node ID.
-     * @param req Request.
-     */
-    private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) {
-        singleNodeId = nodeId;
-        singleReq = req;
-
-        if (cctx.localNodeId().equals(nodeId)) {
-            cache.updateAllAsyncInternal(nodeId, req,
-                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
-                    @Override public void apply(GridNearAtomicUpdateRequest req,
-                        GridNearAtomicUpdateResponse res) {
-                        assert res.futureVersion().equals(futVer) : futVer;
+        /**
+         * @return Request.
+         * @throws Exception If failed.
+         */
+        private GridNearAtomicUpdateRequest mapSingleUpdate() throws Exception {
+            Object key = F.first(keys);
 
-                        onResult(res.nodeId(), res);
-                    }
-                });
-        }
-        else {
-            try {
-                if (log.isDebugEnabled())
-                    log.debug("Sending near atomic update request [nodeId=" + req.nodeId() + ", req=" + req + ']');
+            Object val;
+            GridCacheVersion conflictVer;
+            long conflictTtl;
+            long conflictExpireTime;
 
-                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
+            if (vals != null) {
+                // Regular PUT.
+                val = F.first(vals);
+                conflictVer = null;
+                conflictTtl = CU.TTL_NOT_CHANGED;
+                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
+            }
+            else if (conflictPutVals != null) {
+                // Conflict PUT.
+                GridCacheDrInfo conflictPutVal = F.first(conflictPutVals);
 
-                if (syncMode == FULL_ASYNC && cctx.config().getAtomicWriteOrderMode() == PRIMARY)
-                    onDone(new GridCacheReturn(cctx, true, null, true));
+                val = conflictPutVal.value();
+                conflictVer = conflictPutVal.version();
+                conflictTtl = conflictPutVal.ttl();
+                conflictExpireTime = conflictPutVal.expireTime();
             }
-            catch (IgniteCheckedException e) {
-                onDone(addFailedKeys(req.keys(), req.topologyVersion(), e));
+            else if (conflictRmvVals != null) {
+                // Conflict REMOVE.
+                val = null;
+                conflictVer = F.first(conflictRmvVals);
+                conflictTtl = CU.TTL_NOT_CHANGED;
+                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
+            }
+            else {
+                // Regular REMOVE.
+                val = null;
+                conflictVer = null;
+                conflictTtl = CU.TTL_NOT_CHANGED;
+                conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
             }
-        }
-    }
-
-    /**
-     * Sends messages to remote nodes and updates local cache.
-     *
-     * @param mappings Mappings to send.
-     */
-    private void doUpdate(Map<UUID, GridNearAtomicUpdateRequest> mappings) {
-        UUID locNodeId = cctx.localNodeId();
 
-        GridNearAtomicUpdateRequest locUpdate = null;
+            // We still can get here if user pass map with single element.
+            if (key == null)
+                throw new NullPointerException("Null key.");
 
-        // Send messages to remote nodes first, then run local update.
-        for (GridNearAtomicUpdateRequest req : mappings.values()) {
-            if (locNodeId.equals(req.nodeId())) {
-                assert locUpdate == null : "Cannot have more than one local mapping [locUpdate=" + locUpdate +
-                    ", req=" + req + ']';
+            if (val == null && op != GridCacheOperation.DELETE)
+                throw new NullPointerException("Null value.");
 
-                locUpdate = req;
-            }
-            else {
-                try {
-                    if (log.isDebugEnabled())
-                        log.debug("Sending near atomic update request [nodeId=" + req.nodeId() + ", req=" + req + ']');
+            KeyCacheObject cacheKey = cctx.toCacheKeyObject(key);
 
-                    cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
-                }
-                catch (IgniteCheckedException e) {
-                    addFailedKeys(req.keys(), req.topologyVersion(), e);
+            if (op != TRANSFORM)
+                val = cctx.toCacheObject(val);
 
-                    removeMapping(req.nodeId());
-                }
+            ClusterNode primary = cctx.affinity().primary(cacheKey, topVer);
 
-                if (syncMode == PRIMARY_SYNC && !req.hasPrimary())
-                    removeMapping(req.nodeId());
-            }
-        }
+            if (primary == null)
+                throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " +
+                    "left the grid).");
 
-        if (syncMode == FULL_ASYNC)
-            // In FULL_ASYNC mode always return (null, true).
-            opRes = new GridCacheReturn(cctx, true, null, true);
+            GridNearAtomicUpdateRequest req = new GridNearAtomicUpdateRequest(
+                cctx.cacheId(),
+                primary.id(),
+                futVer,
+                fastMap,
+                updVer,
+                topVer,
+                topLocked,
+                syncMode,
+                op,
+                retval,
+                expiryPlc,
+                invokeArgs,
+                filter,
+                subjId,
+                taskNameHash,
+                skipStore,
+                cctx.kernalContext().clientNode());
 
-        if (locUpdate != null) {
-            cache.updateAllAsyncInternal(cctx.localNodeId(), locUpdate,
-                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
-                    @Override public void apply(GridNearAtomicUpdateRequest req,
-                        GridNearAtomicUpdateResponse res) {
-                        assert res.futureVersion().equals(futVer) : futVer;
+            req.addUpdateEntry(cacheKey,
+                val,
+                conflictTtl,
+                conflictExpireTime,
+                conflictVer,
+                true);
 
-                        onResult(res.nodeId(), res);
-                    }
-                });
+            return req;
         }
 
-        checkComplete();
-    }
-
-    /**
-     * Removes mapping from future mappings map.
-     *
-     * @param nodeId Node ID to remove mapping for.
-     */
-    private void removeMapping(UUID nodeId) {
-        mappings.remove(nodeId);
-    }
-
-    /**
-     * @param ret Result from single node.
-     */
-    @SuppressWarnings("unchecked")
-    private synchronized void addInvokeResults(GridCacheReturn ret) {
-        assert op == TRANSFORM : op;
-        assert ret.value() == null || ret.value() instanceof Map : ret.value();
-
-        if (ret.value() != null) {
-            if (opRes != null)
-                opRes.mergeEntryProcessResults(ret);
-            else
-                opRes = ret;
+        /**
+         * @param ret Result from single node.
+         */
+        @SuppressWarnings("unchecked")
+        private void addInvokeResults(GridCacheReturn ret) {
+            assert op == TRANSFORM : op;
+            assert ret.value() == null || ret.value() instanceof Map : ret.value();
+
+            if (ret.value() != null) {
+                if (opRes != null)
+                    opRes.mergeEntryProcessResults(ret);
+                else
+                    opRes = ret;
+            }
         }
-    }
 
-    /**
-     * @param failedKeys Failed keys.
-     * @param topVer Topology version for failed update.
-     * @param err Error cause.
-     * @return Root {@link org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException}.
-     */
-    private synchronized IgniteCheckedException addFailedKeys(Collection<KeyCacheObject> failedKeys,
-        AffinityTopologyVersion topVer,
-        Throwable err) {
-        CachePartialUpdateCheckedException err0 = this.err;
+        /**
+         * @param failedKeys Failed keys.
+         * @param topVer Topology version for failed update.
+         * @param err Error cause.
+         */
+        private void addFailedKeys(Collection<KeyCacheObject> failedKeys,
+            AffinityTopologyVersion topVer,
+            Throwable err) {
+            CachePartialUpdateCheckedException err0 = this.err;
 
-        if (err0 == null)
-            err0 = this.err = new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
+            if (err0 == null)
+                err0 = this.err = new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
 
-        Collection<Object> keys = new ArrayList<>(failedKeys.size());
+            Collection<Object> keys = new ArrayList<>(failedKeys.size());
 
-        for (KeyCacheObject key : failedKeys)
-            keys.add(key.value(cctx.cacheObjectContext(), false));
+            for (KeyCacheObject key : failedKeys)
+                keys.add(key.value(cctx.cacheObjectContext(), false));
 
-        err0.add(keys, err, topVer);
+            err0.add(keys, err, topVer);
+        }
 
-        return err0;
+        /** {@inheritDoc} */
+        @Override public synchronized  String toString() {
+            return S.toString(UpdateState.class, this);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index 2064338..d56ed7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -128,7 +128,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     /**
      * @param ignoreChildFailures Flag indicating whether compound future should ignore child futures failures.
      */
-    public void ignoreChildFailures(Class<? extends Throwable>... ignoreChildFailures) {
+    @SafeVarargs
+    public final void ignoreChildFailures(Class<? extends Throwable>... ignoreChildFailures) {
         this.ignoreChildFailures = ignoreChildFailures;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
index d5c5314..fc9dad0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
@@ -421,6 +421,7 @@ public final class X {
      * @return {@code True} if one of the causing exception is an instance of passed in classes,
      *      {@code false} otherwise.
      */
+    @SafeVarargs
     public static boolean hasCause(@Nullable Throwable t, @Nullable Class<? extends Throwable>... cls) {
         if (t == null || F.isEmpty(cls))
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerEagerTtlDisabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerEagerTtlDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerEagerTtlDisabledTest.java
index f681e59..97590b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerEagerTtlDisabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerEagerTtlDisabledTest.java
@@ -18,8 +18,10 @@
 package org.apache.ignite.internal.processors.cache;
 
 
+import org.apache.ignite.configuration.*;
+
 /**
- * Tests expire events when {@link GridCacheConfiguration#isEagerTtl()} is disabled.
+ * Tests expire events when {@link CacheConfiguration#isEagerTtl()} is disabled.
  */
 public class IgniteCacheEntryListenerEagerTtlDisabledTest extends IgniteCacheEntryListenerTxTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 5d0cacc..caf4699 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -27,12 +27,14 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
 
+import javax.cache.processor.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
 
 /**
  *
@@ -46,7 +48,9 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     /**
      * @return Keys count for the test.
      */
-    protected abstract int keysCount();
+    private int keysCount() {
+        return 10_000;
+    }
 
     /** {@inheritDoc} */
     @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
@@ -54,7 +58,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
 
         cfg.setAtomicWriteOrderMode(writeOrderMode());
         cfg.setBackups(1);
-        cfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cfg.setRebalanceMode(SYNC);
 
         return cfg;
     }
@@ -78,25 +82,47 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     protected CacheAtomicWriteOrderMode writeOrderMode() {
         return CLOCK;
     }
+
     /**
      * @throws Exception If failed.
      */
     public void testPut() throws Exception {
-        checkPut(false);
+        checkRetry(Test.PUT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAll() throws Exception {
+        checkRetry(Test.PUT_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAsync() throws Exception {
-        checkPut(true);
+        checkRetry(Test.PUT_ASYNC);
     }
 
     /**
-     * @param async If {@code true} tests asynchronous put.
      * @throws Exception If failed.
      */
-    private void checkPut(boolean async) throws Exception {
+    public void testInvoke() throws Exception {
+        checkRetry(Test.INVOKE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAll() throws Exception {
+        checkRetry(Test.INVOKE_ALL);
+    }
+
+    /**
+     * @param test Test type.
+     * @throws Exception If failed.
+     */
+    private void checkRetry(Test test) throws Exception {
         final AtomicBoolean finished = new AtomicBoolean();
 
         int keysCnt = keysCount();
@@ -115,52 +141,151 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
             }
         });
 
+        IgniteCache<Integer, Integer> cache = ignite(0).cache(null);
 
-        IgniteCache<Object, Object> cache = ignite(0).cache(null);
+        int iter = 0;
 
-        if (atomicityMode() == ATOMIC)
-            assertEquals(writeOrderMode(), cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+        try {
+            if (atomicityMode() == ATOMIC)
+                assertEquals(writeOrderMode(), cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
 
-        int iter = 0;
+            long stopTime = System.currentTimeMillis() + 60_000;
 
-        long stopTime = System.currentTimeMillis() + 60_000;
+            switch (test) {
+                case PUT: {
+                    while (System.currentTimeMillis() < stopTime) {
+                        Integer val = ++iter;
 
-        if (async) {
-            IgniteCache<Object, Object> cache0 = cache.withAsync();
+                        for (int i = 0; i < keysCnt; i++)
+                            cache.put(i, val);
 
-            while (System.currentTimeMillis() < stopTime) {
-                Integer val = ++iter;
+                        for (int i = 0; i < keysCnt; i++)
+                            assertEquals(val, cache.get(i));
+                    }
 
-                for (int i = 0; i < keysCnt; i++) {
-                    cache0.put(i, val);
+                    break;
+                }
+
+                case PUT_ALL: {
+                    while (System.currentTimeMillis() < stopTime) {
+                        Integer val = ++iter;
+
+                        Map<Integer, Integer> map = new LinkedHashMap<>();
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            map.put(i, val);
+
+                            if (map.size() == 100 || i == keysCnt - 1) {
+                                cache.putAll(map);
+
+                                map.clear();
+                            }
+                        }
 
-                    cache0.future().get();
+                        for (int i = 0; i < keysCnt; i++)
+                            assertEquals(val, cache.get(i));
+                    }
                 }
 
-                for (int i = 0; i < keysCnt; i++) {
-                    cache0.get(i);
+                case PUT_ASYNC: {
+                    IgniteCache<Integer, Integer> cache0 = cache.withAsync();
+
+                    while (System.currentTimeMillis() < stopTime) {
+                        Integer val = ++iter;
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            cache0.put(i, val);
+
+                            cache0.future().get();
+                        }
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            cache0.get(i);
+
+                            assertEquals(val, cache0.future().get());
+                        }
+                    }
 
-                    assertEquals(val, cache0.future().get());
+                    break;
                 }
-            }
-        }
-        else {
-            while (System.currentTimeMillis() < stopTime) {
-                Integer val = ++iter;
 
-                for (int i = 0; i < keysCnt; i++)
-                    cache.put(i, val);
+                case INVOKE: {
+                    while (System.currentTimeMillis() < stopTime) {
+                        Integer val = ++iter;
+
+                        Integer expOld = iter - 1;
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            Integer old = cache.invoke(i, new SetEntryProcessor(val));
+
+                            assertNotNull(old);
+                            assertTrue(old.equals(expOld) || old.equals(val));
+                        }
+
+                        for (int i = 0; i < keysCnt; i++)
+                            assertEquals(val, cache.get(i));
+                    }
+
+                    break;
+                }
 
-                for (int i = 0; i < keysCnt; i++)
-                    assertEquals(val, cache.get(i));
+                case INVOKE_ALL: {
+                    while (System.currentTimeMillis() < stopTime) {
+                        Integer val = ++iter;
+
+                        Integer expOld = iter - 1;
+
+                        Set<Integer> keys = new LinkedHashSet<>();
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            keys.add(i);
+
+                            if (keys.size() == 100 || i == keysCnt - 1) {
+                                Map<Integer, EntryProcessorResult<Integer>> resMap =
+                                    cache.invokeAll(keys, new SetEntryProcessor(val));
+
+                                for (Integer key : keys) {
+                                    EntryProcessorResult<Integer> res = resMap.get(key);
+
+                                    assertNotNull(res);
+
+                                    Integer old = res.get();
+
+                                    assertTrue(old.equals(expOld) || old.equals(val));
+                                }
+
+                                assertEquals(keys.size(), resMap.size());
+
+                                keys.clear();
+                            }
+                        }
+
+                        for (int i = 0; i < keysCnt; i++)
+                            assertEquals(val, cache.get(i));
+                    }
+
+                    break;
+                }
+
+                default:
+                    assert false : test;
             }
         }
-
-        finished.set(true);
-        fut.get();
+        finally {
+            finished.set(true);
+            fut.get();
+        }
 
         for (int i = 0; i < keysCnt; i++)
-            assertEquals(iter, cache.get(i));
+            assertEquals((Integer)iter, cache.get(i));
+
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteKernal ignite = (IgniteKernal)grid(i);
+
+            Collection<?> futs = ignite.context().cache().context().mvcc().atomicFutures();
+
+            assertTrue("Unexpected atomic futures: " + futs, futs.isEmpty());
+        }
     }
 
     /**
@@ -201,34 +326,41 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
         try {
             int keysCnt = keysCount();
 
-        boolean eThrown = false;
+            boolean eThrown = false;
 
-        IgniteCache<Object, Object> cache = ignite(0).cache(null).withNoRetries();
+            IgniteCache<Object, Object> cache = ignite(0).cache(null).withNoRetries();
 
-        if (async)
-            cache = cache.withAsync();
+            if (async)
+                cache = cache.withAsync();
 
-        for (int i = 0; i < keysCnt; i++) {
-            try {
-                if (async) {
-                    cache.put(i, i);
+            long stopTime = System.currentTimeMillis() + 60_000;
 
-                    cache.future().get();
+            while (System.currentTimeMillis() < stopTime) {
+                for (int i = 0; i < keysCnt; i++) {
+                    try {
+                        if (async) {
+                            cache.put(i, i);
+
+                            cache.future().get();
+                        }
+                        else
+                            cache.put(i, i);
+                    }
+                    catch (Exception e) {
+                        assertTrue("Invalid exception: " + e,
+                            X.hasCause(e, ClusterTopologyCheckedException.class, CachePartialUpdateException.class));
+
+                        eThrown = true;
+
+                        break;
+                    }
                 }
-                else
-                    cache.put(i, i);
-            }
-            catch (Exception e) {
-                assertTrue("Invalid exception: " + e,
-                    X.hasCause(e, ClusterTopologyCheckedException.class, CachePartialUpdateException.class));
-
-                eThrown = true;
 
+                if (eThrown)
                     break;
-                }
             }
 
-        assertTrue(eThrown);
+            assertTrue(eThrown);
 
             finished.set(true);
 
@@ -243,4 +375,48 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     @Override protected long getTestTimeout() {
         return 3 * 60 * 1000;
     }
+
+    /**
+     *
+     */
+    enum Test {
+        /** */
+        PUT,
+
+        /** */
+        PUT_ALL,
+
+        /** */
+        PUT_ASYNC,
+
+        /** */
+        INVOKE,
+
+        /** */
+        INVOKE_ALL
+    }
+
+    /**
+     *
+     */
+    class SetEntryProcessor implements CacheEntryProcessor<Integer, Integer, Integer> {
+        /** */
+        private Integer val;
+
+        /**
+         * @param val Value.
+         */
+        public SetEntryProcessor(Integer val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer process(MutableEntry<Integer, Integer> e, Object... args) {
+            Integer old = e.getValue();
+
+            e.setValue(val);
+
+            return old == null ? 0 : old;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b625a3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
index e76663a..be442d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAtomicSelfTest.java
@@ -16,7 +16,21 @@
  */
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  *
@@ -24,11 +38,66 @@ import org.apache.ignite.cache.*;
 public class IgniteCachePutRetryAtomicSelfTest extends IgniteCachePutRetryAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.ATOMIC;
+        return ATOMIC;
     }
 
-    /** {@inheritDoc} */
-    @Override protected int keysCount() {
-        return 60_000;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutInsideTransaction() throws Exception {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName("tx-cache");
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+
+        try (IgniteCache<Integer, Integer> txCache = ignite(0).getOrCreateCache(ccfg)) {
+            final AtomicBoolean finished = new AtomicBoolean();
+
+            IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    while (!finished.get()) {
+                        stopGrid(3);
+
+                        U.sleep(300);
+
+                        startGrid(3);
+                    }
+
+                    return null;
+                }
+            });
+
+            try {
+                IgniteTransactions txs = ignite(0).transactions();
+
+                IgniteCache<Object, Object> cache = ignite(0).cache(null);
+
+                long stopTime = System.currentTimeMillis() + 60_000;
+
+                while (System.currentTimeMillis() < stopTime) {
+                    for (int i = 0; i < 10_000; i++) {
+                        try {
+                            try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                                txCache.put(0, 0);
+
+                                cache.put(i, i);
+
+                                tx.commit();
+                            }
+                        }
+                        catch (IgniteException | CacheException e) {
+                            log.info("Ignore exception: " + e);
+                        }
+                    }
+                }
+
+                finished.set(true);
+
+                fut.get();
+            }
+            finally {
+                finished.set(true);
+            }
+        }
     }
 }


[11/18] ignite git commit: IGNITE-1325: Implemented platform extensions.

Posted by ak...@apache.org.
IGNITE-1325: Implemented platform extensions.


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

Branch: refs/heads/ignite-843
Commit: 3e4052a6d2b50b538585f94443b354d803e235d2
Parents: a457ab1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 13:16:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 13:16:09 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformTarget.java     | 36 +++++++++++++-----
 .../platform/PlatformAbstractTarget.java        | 40 ++++++++++++++++++++
 2 files changed, 67 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3e4052a6/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
index 6a22453..c45bea1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -46,15 +46,6 @@ public interface PlatformTarget {
     public Object inStreamOutObject(int type, long memPtr) throws Exception;
 
     /**
-     * Operation returning result to memory stream.
-     *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
-     * @throws Exception In case of failure.
-     */
-    public void outStream(int type, long memPtr) throws Exception;
-
-    /**
      * Operation accepting one memory stream and returning result to another memory stream.
      *
      * @param type Operation type.
@@ -76,6 +67,33 @@ public interface PlatformTarget {
     public void inObjectStreamOutStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception;
 
     /**
+     * Operation returning long result.
+     *
+     * @param type Operation type.
+     * @return Result.
+     * @throws Exception In case of failure.
+     */
+    public long outLong(int type) throws Exception;
+
+    /**
+     * Operation returning result to memory stream.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @throws Exception In case of failure.
+     */
+    public void outStream(int type, long memPtr) throws Exception;
+
+    /**
+     * Operation returning object result.
+     *
+     * @param type Operation type.
+     * @return Result.
+     * @throws Exception If failed.
+     */
+    public Object outObject(int type) throws Exception;
+
+    /**
      * Start listening for the future.
      *
      * @param futId Future ID.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e4052a6/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 5864a7e..4c01bc0 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -85,6 +85,16 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
+    @Override public long outLong(int type) throws Exception {
+        try {
+            return processOutLong(type);
+        }
+        catch (Exception e) {
+            throw convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void outStream(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PlatformOutputStream out = mem.output();
@@ -101,6 +111,16 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
+    @Override public Object outObject(int type) throws Exception {
+        try {
+            return processOutObject(type);
+        }
+        catch (Exception e) {
+            throw convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
         try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
             PortableRawReaderEx reader = platformCtx.reader(inMem);
@@ -271,6 +291,16 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * Process OUT operation.
      *
      * @param type Type.
+     * @throws IgniteCheckedException In case of exception.
+     */
+    protected long processOutLong(int type) throws IgniteCheckedException {
+        return throwUnsupported(type);
+    }
+
+    /**
+     * Process OUT operation.
+     *
+     * @param type Type.
      * @param writer Portable writer.
      * @throws IgniteCheckedException In case of exception.
      */
@@ -279,6 +309,16 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /**
+     * Process OUT operation.
+     *
+     * @param type Type.
+     * @throws IgniteCheckedException In case of exception.
+     */
+    protected Object processOutObject(int type) throws IgniteCheckedException {
+        return throwUnsupported(type);
+    }
+
+    /**
      * Throw an exception rendering unsupported operation type.
      *
      * @param type Operation type.


[16/18] ignite git commit: Minors.

Posted by ak...@apache.org.
Minors.


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

Branch: refs/heads/ignite-843
Commit: f6ddf4ea7917d7f2a914ea7cea0c89f221373817
Parents: 71334cf
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 16:17:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 16:17:08 2015 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f6ddf4ea/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
index 7af8392..094c6a5 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
@@ -459,7 +459,8 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /// <summary>
-        /// Synchronized stream read operations from underlying memory. This is required when 
+        /// Synchronized stream read operations from underlying memory. This is required when stream was passed 
+        /// to Java and something might have been written there.
         /// </summary>
         public void SynchronizeInput()
         {


[14/18] ignite git commit: altReleaseDeploymentRepository supported only and 2.8+

Posted by ak...@apache.org.
altReleaseDeploymentRepository supported only and 2.8+


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

Branch: refs/heads/ignite-843
Commit: c2cda00580de7036ab3f2adaf23616dffd10968c
Parents: 4369c20
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Aug 31 14:26:44 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Aug 31 14:26:44 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/pom.xml | 13 ++++++++++++-
 1 file changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c2cda005/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index ea95d06..4054004 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -20,7 +20,8 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -32,4 +33,14 @@
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
     <version>1.4.1-SNAPSHOT</version>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <version>2.8.2</version>
+            </plugin>
+        </plugins>
+    </build>
 </project>


[18/18] ignite git commit: Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843

Posted by ak...@apache.org.
Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843


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

Branch: refs/heads/ignite-843
Commit: 1f165949240d738037acac16212da20c23c536ed
Parents: 09d3a7b 1335580
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Aug 31 21:10:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Aug 31 21:10:23 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   22 +-
 DISCLAIMER.txt                                  |   15 -
 LICENSE                                         |    2 +-
 NOTICE                                          |    2 +-
 assembly/LICENSE_FABRIC                         |    2 +-
 assembly/LICENSE_HADOOP                         |    2 +-
 assembly/NOTICE_FABRIC                          |    2 +-
 assembly/NOTICE_HADOOP                          |    2 +-
 assembly/release-base.xml                       |    5 -
 dev-tools/slurp.sh                              |    2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |    2 +-
 modules/apache-license-gen/pom.xml              |   13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |    2 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |    2 +-
 modules/core/src/main/java/META-INF/LICENSE     |    2 +-
 modules/core/src/main/java/META-INF/NOTICE      |    2 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |    2 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |    2 +-
 .../configuration/IgniteReflectionFactory.java  |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |    6 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../discovery/GridDiscoveryManager.java         |   13 +-
 .../processors/cache/GridCacheAtomicFuture.java |    5 -
 .../processors/cache/GridCacheIoManager.java    |    6 +-
 .../processors/cache/GridCacheMvccManager.java  |   62 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    6 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    7 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 1235 +++++++++---------
 .../platform/PlatformNoopProcessor.java         |   66 +
 .../processors/platform/PlatformProcessor.java  |  111 +-
 .../processors/platform/PlatformTarget.java     |   74 +-
 .../util/future/GridCompoundFuture.java         |    3 +-
 .../apache/ignite/internal/util/typedef/X.java  |    1 +
 .../apache/ignite/marshaller/Marshaller.java    |    2 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    2 +-
 .../optimized/OptimizedMarshaller.java          |    2 +-
 .../marshaller/portable/PortableMarshaller.java |    5 +-
 .../SpringApplicationContextResource.java       |    2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |    2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |    2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |    2 +-
 .../jobstealing/JobStealingCollisionSpi.java    |    4 +-
 .../PriorityQueueCollisionSpi.java              |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +-
 .../memory/MemoryEventStorageSpi.java           |    2 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |    2 +-
 .../jobstealing/JobStealingFailoverSpi.java     |    2 +-
 .../spi/failover/never/NeverFailoverSpi.java    |    2 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |    2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |    2 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |    2 +-
 .../WeightedRandomLoadBalancingSpi.java         |    2 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    2 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   50 +
 ...eCacheEntryListenerEagerTtlDisabledTest.java |    4 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |  282 +++-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |   77 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   14 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |    5 -
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |    9 +-
 modules/docker/download_ignite.sh               |    2 +-
 .../CacheHibernateBlobStoreFactory.java         |    2 +-
 .../log4j2/GridLog4j2InitializedTest.java       |    6 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    3 +-
 .../platform/PlatformAbstractTarget.java        |  141 +-
 .../platform/cache/PlatformCache.java           |  135 +-
 .../platform/cache/PlatformCacheIterator.java   |    4 +-
 .../cache/affinity/PlatformAffinity.java        |   10 +-
 .../query/PlatformAbstractQueryCursor.java      |    4 +-
 .../platform/cluster/PlatformClusterGroup.java  |   21 +-
 .../platform/compute/PlatformCompute.java       |   11 +-
 .../datastreamer/PlatformDataStreamer.java      |    4 +-
 .../platform/events/PlatformEvents.java         |   20 +-
 .../platform/messaging/PlatformMessaging.java   |   11 +-
 .../platform/services/PlatformServices.java     |   33 +-
 .../transactions/PlatformTransactions.java      |    4 +-
 .../rest/protocols/http/jetty/rest.html         |    4 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 .../ant/beautifier/GridJavadocAntTask.java      |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 modules/yardstick/README.txt                    |    2 +-
 parent/pom.xml                                  |   12 +-
 84 files changed, 1573 insertions(+), 1035 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f165949/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f165949/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f165949/parent/pom.xml
----------------------------------------------------------------------


[05/18] ignite git commit: MInors.

Posted by ak...@apache.org.
MInors.


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

Branch: refs/heads/ignite-843
Commit: a457ab13471671a3177718a80aed660c837fcfb7
Parents: 0995037
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 11:51:02 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 11:51:02 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/platform/cache/PlatformCache.java    | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a457ab13/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index 0347d4d..e0cbcbe 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -680,7 +680,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      * {@link javax.cache.integration.CacheWriter}s.
      *
      * @throws IllegalStateException if the cache is closed.
-     * @throws javax.cache.CacheException        if there is a problem during the clear
+     * @throws javax.cache.CacheException if there is a problem during the clear
      */
     public void clear() throws IgniteCheckedException {
         cache.clear();
@@ -819,8 +819,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Runs specified query.
      */
-    private PlatformQueryCursor runQuery(PortableRawReaderEx reader, Query qry)
-        throws IgniteCheckedException {
+    private PlatformQueryCursor runQuery(PortableRawReaderEx reader, Query qry) throws IgniteCheckedException {
 
         try {
             QueryCursorEx cursor = (QueryCursorEx) cache.query(qry);
@@ -852,8 +851,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads the query of specified type.
      */
-    private Query readInitialQuery(PortableRawReaderEx reader)
-        throws IgniteCheckedException {
+    private Query readInitialQuery(PortableRawReaderEx reader) throws IgniteCheckedException {
         int typ = reader.readInt();
 
         switch (typ) {


[12/18] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: c2de38d13a1b116cf923125416e3e894604d8b01
Parents: 3e4052a 4369c20
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 13:16:47 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 13:16:47 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    | 22 ++++++++++----------
 DISCLAIMER.txt                                  | 15 -------------
 LICENSE                                         |  2 +-
 NOTICE                                          |  2 +-
 assembly/LICENSE_FABRIC                         |  2 +-
 assembly/LICENSE_HADOOP                         |  2 +-
 assembly/NOTICE_FABRIC                          |  2 +-
 assembly/NOTICE_HADOOP                          |  2 +-
 assembly/release-base.xml                       |  5 -----
 dev-tools/slurp.sh                              |  2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  2 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |  2 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |  2 +-
 modules/core/src/main/java/META-INF/LICENSE     |  2 +-
 modules/core/src/main/java/META-INF/NOTICE      |  2 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |  2 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |  2 +-
 .../configuration/IgniteReflectionFactory.java  |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  2 +-
 .../processors/cache/GridCacheIoManager.java    |  6 +++++-
 .../apache/ignite/marshaller/Marshaller.java    |  2 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  2 +-
 .../optimized/OptimizedMarshaller.java          |  2 +-
 .../marshaller/portable/PortableMarshaller.java |  2 +-
 .../SpringApplicationContextResource.java       |  2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |  2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |  2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |  2 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  4 ++--
 .../PriorityQueueCollisionSpi.java              |  2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  2 +-
 .../memory/MemoryEventStorageSpi.java           |  2 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  2 +-
 .../jobstealing/JobStealingFailoverSpi.java     |  2 +-
 .../spi/failover/never/NeverFailoverSpi.java    |  2 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |  2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  2 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  2 +-
 .../WeightedRandomLoadBalancingSpi.java         |  2 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  2 +-
 modules/docker/download_ignite.sh               |  2 +-
 .../CacheHibernateBlobStoreFactory.java         |  2 +-
 .../log4j2/GridLog4j2InitializedTest.java       |  6 +++---
 .../rest/protocols/http/jetty/rest.html         |  4 ++--
 modules/scalar-2.10/pom.xml                     |  2 +-
 modules/scalar/pom.xml                          |  2 +-
 .../ant/beautifier/GridJavadocAntTask.java      |  2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |  2 +-
 modules/yardstick/README.txt                    |  2 +-
 parent/pom.xml                                  | 12 +++++------
 51 files changed, 72 insertions(+), 88 deletions(-)
----------------------------------------------------------------------



[08/18] ignite git commit: Ignite-1330

Posted by ak...@apache.org.
Ignite-1330


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

Branch: refs/heads/ignite-843
Commit: a95c8622395d3b4ca57e38174c870f9b114497c8
Parents: e6de574
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Aug 31 13:02:45 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Aug 31 13:02:45 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    | 22 ++++++++++----------
 DISCLAIMER.txt                                  | 15 -------------
 LICENSE                                         |  2 +-
 NOTICE                                          |  2 +-
 assembly/LICENSE_FABRIC                         |  2 +-
 assembly/LICENSE_HADOOP                         |  2 +-
 assembly/NOTICE_FABRIC                          |  2 +-
 assembly/NOTICE_HADOOP                          |  2 +-
 dev-tools/slurp.sh                              |  2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  2 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |  2 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |  2 +-
 modules/core/src/main/java/META-INF/LICENSE     |  2 +-
 modules/core/src/main/java/META-INF/NOTICE      |  2 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |  2 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |  2 +-
 .../configuration/IgniteReflectionFactory.java  |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  2 +-
 .../apache/ignite/marshaller/Marshaller.java    |  2 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  2 +-
 .../optimized/OptimizedMarshaller.java          |  2 +-
 .../marshaller/portable/PortableMarshaller.java |  2 +-
 .../SpringApplicationContextResource.java       |  2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |  2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |  2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |  2 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  4 ++--
 .../PriorityQueueCollisionSpi.java              |  2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  2 +-
 .../memory/MemoryEventStorageSpi.java           |  2 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  2 +-
 .../jobstealing/JobStealingFailoverSpi.java     |  2 +-
 .../spi/failover/never/NeverFailoverSpi.java    |  2 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |  2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  2 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  2 +-
 .../WeightedRandomLoadBalancingSpi.java         |  2 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  2 +-
 modules/docker/download_ignite.sh               |  2 +-
 .../CacheHibernateBlobStoreFactory.java         |  2 +-
 .../log4j2/GridLog4j2InitializedTest.java       |  6 +++---
 .../rest/protocols/http/jetty/rest.html         |  4 ++--
 modules/scalar-2.10/pom.xml                     |  2 +-
 modules/scalar/pom.xml                          |  2 +-
 .../ant/beautifier/GridJavadocAntTask.java      |  2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |  2 +-
 modules/yardstick/README.txt                    |  2 +-
 parent/pom.xml                                  | 12 +++++------
 49 files changed, 67 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index b5e556a..a46b73d 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -12,7 +12,7 @@ With LGPL dependencies:
 With LGPL dependencies and Scala 2.10:
   mvn clean package -DskipTests -Prelease,lgpl -Dscala-2.10
 
-Look for incubator-ignite-<version>-bin.zip in ./target/bin directory.
+Look for apache-ignite-fabric-<version>-bin.zip in ./target/bin directory.
 
 NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
       Schema Import module use JavaFX, under OpenJDK may be compiled and run only on OpenJDK 8 with installed OpenJFX.
@@ -23,7 +23,7 @@ mvn clean package -DskipTests -Dignite.edition=hadoop [-Dhadoop.version=X.X.X]
 
 Use 'hadoop.version' parameter to build Ignite against a specific Hadoop version.
 
-Look for incubator-ignite-<version>-bin.zip in ./target/bin directory.
+Look for apache-ignite-hadoop-<version>-bin.zip in ./target/bin directory.
 
 NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 
@@ -31,7 +31,7 @@ NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 Ignite Release Instructions
 ===========================
 Use your people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
-Download https://dist.apache.org/repos/dist/release/incubator/ignite/KEYS and append you key using commands:
+Download https://dist.apache.org/repos/dist/release/ignite/KEYS and append you key using commands:
 
 gpg --list-sigs <keyname> >> KEYS
 gpg --armor --export <keyname> >> KEYS
@@ -68,21 +68,21 @@ Deploy Ignite release candidate to maven repository and dev-svn, make tag:
    Nexus staging (repository.apache.org) should be closed with appropriate comment contains release version and
    release candidate number, for example "Apache Ignite 1.0.0-rc7", when mvn deploy finished.
 
-   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create release candidate folder with name
+   Checkout https://dist.apache.org/repos/dist/dev/ignite svn. Create release candidate folder with name
    equals to release version with "-rc*" ending, for example "1.0.0-rc7", at svn root.
    Copy /target/site folder content to svn/ignite/<rc-version> folder and commit with appropriate comment.
 
-   Make appropriate git tag for release candidate, for example "ignite-1.0.0-incubating-rc7".
+   Make appropriate git tag for release candidate, for example "ignite-X.Y.Z-rc1".
 
-Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>.
+Start vote based on https://dist.apache.org/repos/dist/dev/ignite/<rc-version>.
 
-Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>
-to https://dist.apache.org/repos/dist/release/incubator/ignite/<version> when version accepted.
-Use svn mv ^/dev/incubator/ignite/<rc-version> ^/release/incubator/ignite/<version> command for proper moving.
+Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/ignite/<rc-version>
+to https://dist.apache.org/repos/dist/release/ignite/<version> when version accepted.
+Use svn mv ^/dev/ignite/<rc-version> ^/release/ignite/<version> command for proper moving.
 
-Make appropriate git tag for released version, for example "ignite-1.0.0-incubating".
+Make appropriate git tag for released version, for example "ignite-X.Y.Z".
 
-Send an email to dev@ignite.incubator.apache.org contains release svn url.
+Send an email to dev@ignite.apache.org contains release svn url.
 
 
 JCache TCK compliance

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/DISCLAIMER.txt
----------------------------------------------------------------------
diff --git a/DISCLAIMER.txt b/DISCLAIMER.txt
deleted file mode 100644
index 734f01a..0000000
--- a/DISCLAIMER.txt
+++ /dev/null
@@ -1,15 +0,0 @@
-Apache Ignite is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
-
-For more information about the incubation status of the Apache Ignite project
-you can go to the following page:
-
-http://ignite.incubator.apache.org/

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 7649b39..c971434 100644
--- a/LICENSE
+++ b/LICENSE
@@ -204,7 +204,7 @@
 
 
 ==============================================================================
-Apache Ignite (incubating) Subcomponents:
+Apache Ignite Subcomponents:
 
 The Apache Ignite project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 298d05b..5ec3b0e 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,4 +1,4 @@
-Apache Ignite (incubating)
+Apache Ignite
 Copyright 2015 The Apache Software Foundation
 
 This product includes software developed at

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/assembly/LICENSE_FABRIC
----------------------------------------------------------------------
diff --git a/assembly/LICENSE_FABRIC b/assembly/LICENSE_FABRIC
index 152cc1e..3f92502 100644
--- a/assembly/LICENSE_FABRIC
+++ b/assembly/LICENSE_FABRIC
@@ -204,7 +204,7 @@
 
 
 ==============================================================================
-Apache Ignite (incubating) Subcomponents:
+Apache Ignite Subcomponents:
 
 The Apache Ignite project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/assembly/LICENSE_HADOOP
----------------------------------------------------------------------
diff --git a/assembly/LICENSE_HADOOP b/assembly/LICENSE_HADOOP
index cfbaea3..8d4cc62 100644
--- a/assembly/LICENSE_HADOOP
+++ b/assembly/LICENSE_HADOOP
@@ -204,7 +204,7 @@
 
 
 ==============================================================================
-Apache Ignite (incubating) Subcomponents:
+Apache Ignite Subcomponents:
 
 The Apache Ignite project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/assembly/NOTICE_FABRIC
----------------------------------------------------------------------
diff --git a/assembly/NOTICE_FABRIC b/assembly/NOTICE_FABRIC
index 1c6bd92..2e55768 100644
--- a/assembly/NOTICE_FABRIC
+++ b/assembly/NOTICE_FABRIC
@@ -1,4 +1,4 @@
-Apache Ignite (incubating)
+Apache Ignite
 Copyright 2015 The Apache Software Foundation
 
 This product includes software developed at

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/assembly/NOTICE_HADOOP
----------------------------------------------------------------------
diff --git a/assembly/NOTICE_HADOOP b/assembly/NOTICE_HADOOP
index 298d05b..5ec3b0e 100644
--- a/assembly/NOTICE_HADOOP
+++ b/assembly/NOTICE_HADOOP
@@ -1,4 +1,4 @@
-Apache Ignite (incubating)
+Apache Ignite
 Copyright 2015 The Apache Software Foundation
 
 This product includes software developed at

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/dev-tools/slurp.sh
----------------------------------------------------------------------
diff --git a/dev-tools/slurp.sh b/dev-tools/slurp.sh
index ad7bc01..859e735 100755
--- a/dev-tools/slurp.sh
+++ b/dev-tools/slurp.sh
@@ -56,7 +56,7 @@ echo "<"$(date + "%D - %H:%M:%S")"> Starting task triggering"
 echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>"
 
 # Useful settings
-#cd /home/teamcity/jobs/incubator-ignite/
+#cd /home/teamcity/jobs/ignite/
 #
 #export JAVA_HOME=<java_home>
 #export PATH=$PATH:<gradle_path>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
index 8498cf0..e5f9129 100644
--- a/dev-tools/src/main/groovy/jiraslurp.groovy
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -136,7 +136,7 @@ def sendGetRequest = { urlString, user, pwd->
  * Parsing a special filter from Apache Ignite JIRA and picking up latest by ID
  * attachments to process.
  */
-final GIT_REPO = "https://git1-us-west.apache.org/repos/asf/incubator-ignite.git"
+final GIT_REPO = "https://git1-us-west.apache.org/repos/asf/ignite.git"
 final JIRA_URL = "https://issues.apache.org"
 final ATTACHMENT_URL = "$JIRA_URL/jira/secure/attachment"
 final HISTORY_FILE = "${System.getProperty("user.home")}/validated-jira.txt"

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
index c5a428a..09671bd 100644
--- a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
+++ b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpi.java
@@ -94,7 +94,7 @@ import java.util.*;
  * Choose another implementation of {@link org.apache.ignite.spi.checkpoint.CheckpointSpi} for local or
  * home network tests.
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.checkpoint.CheckpointSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/cloud/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/cloud/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinder.java b/modules/cloud/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinder.java
index 7555b16..e2b0283 100644
--- a/modules/cloud/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinder.java
+++ b/modules/cloud/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinder.java
@@ -115,7 +115,7 @@ import java.util.concurrent.atomic.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/META-INF/LICENSE b/modules/core/src/main/java/META-INF/LICENSE
index 7649b39..c971434 100644
--- a/modules/core/src/main/java/META-INF/LICENSE
+++ b/modules/core/src/main/java/META-INF/LICENSE
@@ -204,7 +204,7 @@
 
 
 ==============================================================================
-Apache Ignite (incubating) Subcomponents:
+Apache Ignite Subcomponents:
 
 The Apache Ignite project contains subcomponents with separate copyright
 notices and license terms. Your use of the source code for the these

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/META-INF/NOTICE b/modules/core/src/main/java/META-INF/NOTICE
index 298d05b..5ec3b0e 100644
--- a/modules/core/src/main/java/META-INF/NOTICE
+++ b/modules/core/src/main/java/META-INF/NOTICE
@@ -1,4 +1,4 @@
-Apache Ignite (incubating)
+Apache Ignite
 Copyright 2015 The Apache Software Foundation
 
 This product includes software developed at

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
index cf493a0..864c3aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
@@ -55,7 +55,7 @@ import javax.sql.*;
  *     &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
index 5449433..7e7a248 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
@@ -54,7 +54,7 @@ import javax.sql.*;
  *     &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
index af73737..d4569d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
@@ -88,7 +88,7 @@ import java.util.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see Factory

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 1db73bf..489973f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -103,7 +103,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** Ignite site that is shown in log messages. */
-    static final String SITE = "ignite.incubator.apache.org";
+    static final String SITE = "ignite.apache.org";
 
     /** System line separator. */
     private static final String NL = U.nl();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/marshaller/Marshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/Marshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/Marshaller.java
index c25e74e..c0db6b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/Marshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/Marshaller.java
@@ -60,7 +60,7 @@ import java.io.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
index 8a61c41..9cff3b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
@@ -55,7 +55,7 @@ import java.io.*;
  * &lt;/bean&gt;
  * </pre>
  *  <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
index accfeb7..338d277 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
@@ -71,7 +71,7 @@ import java.util.concurrent.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index 2a17363..2cac90e 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -63,7 +63,7 @@ import java.util.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/resources/SpringApplicationContextResource.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/resources/SpringApplicationContextResource.java b/modules/core/src/main/java/org/apache/ignite/resources/SpringApplicationContextResource.java
index 38c0c88..5f41ee1 100644
--- a/modules/core/src/main/java/org/apache/ignite/resources/SpringApplicationContextResource.java
+++ b/modules/core/src/main/java/org/apache/ignite/resources/SpringApplicationContextResource.java
@@ -56,7 +56,7 @@ import java.lang.annotation.*;
  * }
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/cache/CacheCheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/cache/CacheCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/cache/CacheCheckpointSpi.java
index db6d1b0..16a7c02 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/cache/CacheCheckpointSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/cache/CacheCheckpointSpi.java
@@ -88,7 +88,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.checkpoint.CheckpointSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/jdbc/JdbcCheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/jdbc/JdbcCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/jdbc/JdbcCheckpointSpi.java
index f25c46d..6ab9e95 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/jdbc/JdbcCheckpointSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/jdbc/JdbcCheckpointSpi.java
@@ -97,7 +97,7 @@ import java.text.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
index 21880de..9cd4a94 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
@@ -93,7 +93,7 @@ import java.util.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.checkpoint.CheckpointSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
index 8a397ce..9055b98 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingCollisionSpi.java
@@ -58,7 +58,7 @@ import static org.apache.ignite.events.EventType.*;
  * when Node<sub>3</sub> becomes free, it steals Job<sub>13</sub> and Job<sub>23</sub>
  * from Node<sub>1</sub> and Node<sub>2</sub> respectively.
  * <p>
- * <center><img src="http://http://ignite.incubator.apache.org/images/job_stealing_white.gif"></center>
+ * <center><img src="http://http://ignite.apache.org/images/job_stealing_white.gif"></center>
  * <p>
  * <i>
  * Note that this SPI must always be used in conjunction with
@@ -156,7 +156,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/collision/priorityqueue/PriorityQueueCollisionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/priorityqueue/PriorityQueueCollisionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/priorityqueue/PriorityQueueCollisionSpi.java
index 24da2f9..3b71331 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/collision/priorityqueue/PriorityQueueCollisionSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/priorityqueue/PriorityQueueCollisionSpi.java
@@ -153,7 +153,7 @@ import java.util.*;
  * }
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 4022cf6..25b219b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -149,7 +149,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see CommunicationSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index e683dad..a4947f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -156,7 +156,7 @@ import java.util.concurrent.atomic.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see DiscoverySpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/eventstorage/memory/MemoryEventStorageSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/eventstorage/memory/MemoryEventStorageSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/eventstorage/memory/MemoryEventStorageSpi.java
index 46029a9..ff16e47 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/eventstorage/memory/MemoryEventStorageSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/eventstorage/memory/MemoryEventStorageSpi.java
@@ -84,7 +84,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.eventstorage.EventStorageSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
index e925995..5f2bd55 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
@@ -74,7 +74,7 @@ import java.util.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.failover.FailoverSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
index eef1f28..44f8309 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
@@ -80,7 +80,7 @@ import static org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSp
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.failover.FailoverSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/failover/never/NeverFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/never/NeverFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/never/NeverFailoverSpi.java
index 68a24df..fc249db 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/never/NeverFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/never/NeverFailoverSpi.java
@@ -55,7 +55,7 @@ import java.util.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.failover.FailoverSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
index 15c6fe4..18b0c00 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
@@ -54,7 +54,7 @@ import java.util.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
index 8d4d41a..81c576a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
@@ -224,7 +224,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
index 6ae9e99..fd3295f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
@@ -155,7 +155,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
index 6fcce26..864fb24 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
@@ -153,7 +153,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/property&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
index 000782a..d0e67eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -94,7 +94,7 @@ import static org.apache.ignite.events.EventType.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.swapspace.SwapSpaceSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/docker/download_ignite.sh
----------------------------------------------------------------------
diff --git a/modules/docker/download_ignite.sh b/modules/docker/download_ignite.sh
index 9e2cb51..8a91c59 100644
--- a/modules/docker/download_ignite.sh
+++ b/modules/docker/download_ignite.sh
@@ -32,7 +32,7 @@ fi
 
 if [ ! -z $IGNITE_VERSION ]; then
   if [[ $IGNITE_VERSION  =~ [0-9]*\.[0-9]*\.0 ]]; then
-    download http://apache-mirror.rbc.ru/pub/apache/incubator/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-incubating-bin.zip
+    download http://apache-mirror.rbc.ru/pub/apache/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip
   else
     download http://www.gridgain.com/media/gridgain-community-fabric-${IGNITE_VERSION}.zip
   fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
index 60eacb9..4da8eb0 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
@@ -116,7 +116,7 @@ import java.util.*;
  *     ...
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/GridLog4j2InitializedTest.java
----------------------------------------------------------------------
diff --git a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/GridLog4j2InitializedTest.java b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/GridLog4j2InitializedTest.java
index 3059861..b907e10 100644
--- a/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/GridLog4j2InitializedTest.java
+++ b/modules/log4j2/src/test/java/org/apache/ignite/logger/log4j2/GridLog4j2InitializedTest.java
@@ -38,7 +38,7 @@ public class GridLog4j2InitializedTest extends TestCase {
      */
     @Override
     protected void setUp() throws Exception {
-  
+
     }
 
     /** */
@@ -48,8 +48,8 @@ public class GridLog4j2InitializedTest extends TestCase {
 
         cfg.setGridName("grid" + 1);
         cfg.setNodeId(new UUID(1, 1));
-        // cfg.setIgniteHome("/home/glutters/Documenti/apache-ignite/ignite-master/incubator-ignite/");
-       
+        // cfg.setIgniteHome("/home/glutters/Documenti/apache-ignite/ignite-master/ignite/");
+
         URL xml = U.resolveIgniteUrl("config/ignite-log4j2.xml");
         IgniteLogger log;
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/rest.html
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/rest.html b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/rest.html
index b8ad9e0..a487921 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/rest.html
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/rest.html
@@ -55,8 +55,8 @@
     <br>
     <br>
     <center>
-        <a class="img_link" href="https://incubator.apache.org/projects/ignite.html" title="Apache Software Foundation">
-            <img src="http://ignite.incubator.apache.org/images/logo3.png" alt="Ignite - In-Memory Data Fabric">
+        <a class="img_link" href="https://apache.org/projects/ignite.html" title="Apache Software Foundation">
+            <img src="http://ignite.apache.org/images/logo3.png" alt="Ignite - In-Memory Data Fabric">
         </a>
         <p>
         <div style="width: 650px; text-align: justify; padding-top: 20px">

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index a2ae7b8..e21ba47 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -151,7 +151,7 @@
                                             <!--<td>-->
                                                 <!--<nobr>Ignite&#153; - Scalar DSL, ver. <strong>${project.version}</strong></nobr>-->
                                                 <!--<br>-->
-                                                <!--<a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>-->
+                                                <!--<a target=_blank href="https://apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>-->
                                             <!--</td>-->
                                         <!--</tr>-->
                                         <!--</table>-->

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 84a709d..81f4427 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -142,7 +142,7 @@
                                             <td>
                                                 <nobr>Ignite&#153; - Scalar DSL, ver. <strong>${project.version}</strong></nobr>
                                                 <br>
-                                                <a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
+                                                <a target=_blank href="https://apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
                                             </td>
                                         </tr>
                                         </table>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java b/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
index 8ba8c90..8479863 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
@@ -250,7 +250,7 @@ public class GridJavadocAntTask extends MatchingTask {
                 case TOKEN_CLOSE_TAG: {
                     if ("</head>".equalsIgnoreCase(val))
                         tok.update(
-                            "<link rel='shortcut icon' href='https://ignite.incubator.apache.org/favicon.ico'/>\n" +
+                            "<link rel='shortcut icon' href='https://ignite.apache.org/favicon.ico'/>\n" +
                             "<link type='text/css' rel='stylesheet' href='" + SH_URL + "/styles/shCore.css'/>\n" +
                             "<link type='text/css' rel='stylesheet' href='" + SH_URL +
                                 "/styles/shThemeDefault.css'/>\n" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
index 3a397c2..f2918c0 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
@@ -250,7 +250,7 @@ import java.util.Map.*;
  * &lt;/bean&gt;
  * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <img src="http://ignite.apache.org/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  * @see org.apache.ignite.spi.deployment.DeploymentSpi

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/modules/yardstick/README.txt
----------------------------------------------------------------------
diff --git a/modules/yardstick/README.txt b/modules/yardstick/README.txt
index c52838d..8cbe3cf 100644
--- a/modules/yardstick/README.txt
+++ b/modules/yardstick/README.txt
@@ -1,6 +1,6 @@
 Yardstick Ignite Benchmarks
 ===========================
-Yardstick Ignite is a set of Ignite Grid (http://ignite.incubator.apache.org/) benchmarks written on top of Yardstick framework.
+Yardstick Ignite is a set of Ignite Grid (http://ignite.apache.org/) benchmarks written on top of Yardstick framework.
 
 Yardstick Framework
 ===================

http://git-wip-us.apache.org/repos/asf/ignite/blob/a95c8622/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 8d25cf2..a9ebf84 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -49,7 +49,7 @@
     <version>1</version>
     <packaging>pom</packaging>
 
-    <url>http://ignite.incubator.apache.org</url>
+    <url>http://ignite.apache.org</url>
 
     <description>Java-based middleware for in-memory processing of big data in a distributed environment.</description>
 
@@ -63,10 +63,10 @@
     <mailingLists>
         <mailingList>
             <name>Ignite Dev List</name>
-            <subscribe>dev-subscribe@ignite.incubator.apache.org</subscribe>
-            <unsubscribe>dev-unsubscribe@ignite.incubator.apache.org</unsubscribe>
-            <post>dev@ignite.incubator.apache.org</post>
-            <archive>http://mail-archives.apache.org/mod_mbox/incubator-ignite-dev</archive>
+            <subscribe>dev-subscribe@ignite.apache.org</subscribe>
+            <unsubscribe>dev-unsubscribe@ignite.apache.org</unsubscribe>
+            <post>dev@ignite.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/ignite-dev</archive>
         </mailingList>
     </mailingLists>
 
@@ -356,7 +356,7 @@
                                                 <tbody style="padding: 0; margin: 0">
                                                     <tr style="padding: 0; margin: 0">
                                                         <td>
-                                                            <a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
+                                                            <a target=_blank href="https://apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
                                                         </td>
                                                     </tr>
                                                 </tbody>


[13/18] ignite git commit: Defined platform processor interface.

Posted by ak...@apache.org.
Defined platform processor interface.


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

Branch: refs/heads/ignite-843
Commit: 13358c3e03d48f481abe6a7178adbfcec4bd92d9
Parents: c2de38d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 13:40:54 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 13:40:54 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformNoopProcessor.java         |  61 +++++++++++
 .../processors/platform/PlatformProcessor.java  | 106 ++++++++++++++++++-
 2 files changed, 164 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/13358c3e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index 7cb65d3..eadb4c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
+import org.jetbrains.annotations.*;
 
 /**
  * No-op processor.
@@ -45,11 +46,71 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
+    @Override public void releaseStart() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void awaitStart() throws IgniteCheckedException {
         // No-op.
     }
 
     /** {@inheritDoc} */
+    @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepPortable) throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget transactions() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget projection() throws IgniteCheckedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget compute(PlatformTarget grp) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget message(PlatformTarget grp) {
+        return null;
+    }
+
+    @Override
+    public PlatformTarget events(PlatformTarget grp) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget services(PlatformTarget grp) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public PlatformTarget extensions() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/13358c3e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 0cf2df4..a22c787 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -19,13 +19,15 @@ package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.*;
+import org.jetbrains.annotations.*;
 
 /**
  * Platform processor.
  */
+@SuppressWarnings("UnusedDeclaration")
 public interface PlatformProcessor extends GridProcessor {
     /**
-     * Get owning Ignite instance.
+     * Gets owning Ignite instance.
      *
      * @return Ignite instance.
      */
@@ -46,16 +48,114 @@ public interface PlatformProcessor extends GridProcessor {
     public PlatformContext context();
 
     /**
-     * Await until platform processor is safe to use.
+     * Notify processor that it is safe to use.
+     */
+    public void releaseStart();
+
+    /**
+     * Await until platform processor is safe to use (i.e. {@link #releaseStart() has been called}.
      *
      * @throws IgniteCheckedException If failed.
      */
     public void awaitStart() throws IgniteCheckedException;
 
     /**
+     * Get cache.
+     *
+     * @param name Cache name.
+     * @return Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException;
+
+    /**
+     * Create cache.
+     *
+     * @param name Cache name.
+     * @return Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException;
+
+    /**
+     * Get or create cache.
+     *
+     * @param name Cache name.
+     * @return Cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
+
+    /**
+     * Get affinity.
+     *
+     * @param name Cache name.
+     * @return Affinity.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException;
+
+    /**
+     * Get data streamer.
+     *
+     * @param cacheName Cache name.
+     * @param keepPortable Portable flag.
+     * @return Data streamer.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepPortable) throws IgniteCheckedException;
+
+    /**
+     * Get transactions.
+     *
+     * @return Transactions.
+     */
+    public PlatformTarget transactions();
+
+    /**
+     * Get projection.
+     *
+     * @return Projection.
+     * @throws IgniteCheckedException If failed.
+     */
+    public PlatformTarget projection() throws IgniteCheckedException;
+
+    /**
+     * Create interop compute.
+     *
+     * @param grp Cluster group.
+     * @return Compute instance.
+     */
+    public PlatformTarget compute(PlatformTarget grp);
+
+    /**
+     * Create interop messaging.
+     *
+     * @param grp Cluster group.
+     * @return Messaging instance.
+     */
+    public PlatformTarget message(PlatformTarget grp);
+
+    /**
+     * Create interop events.
+     *
+     * @param grp Cluster group.
+     * @return Events instance.
+     */
+    public PlatformTarget events(PlatformTarget grp);
+
+    /**
+     * Create interop services.
+     *
+     * @param grp Cluster group.
+     * @return Services instance.
+     */
+    public PlatformTarget services(PlatformTarget grp);
+
+    /**
      * Get platform extensions. Override this method to provide any additional targets and operations you need.
      *
      * @return Platform extensions.
      */
-    PlatformTarget extensions();
+    public PlatformTarget extensions();
 }


[15/18] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 71334cf9fd893fa45f6dd0bd72a27a96d09fc4e5
Parents: c2cda00 13358c3
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Aug 31 14:27:06 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Aug 31 14:27:06 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformNoopProcessor.java         |  61 +++++++++++
 .../processors/platform/PlatformProcessor.java  | 106 ++++++++++++++++++-
 .../processors/platform/PlatformTarget.java     |  36 +++++--
 .../platform/PlatformAbstractTarget.java        |  40 +++++++
 4 files changed, 231 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[06/18] ignite git commit: Avoid potential blocking of the thread completing affinityReadyFuture.

Posted by ak...@apache.org.
Avoid potential blocking of the thread completing affinityReadyFuture.


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

Branch: refs/heads/ignite-843
Commit: 8a742a2be00b8f701372b87e1782a7348d0ed899
Parents: e6de574
Author: sboikov <sb...@gridgain.com>
Authored: Mon Aug 31 12:10:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Aug 31 12:10:12 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheIoManager.java   | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8a742a2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 26253ed..cfdf06c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -118,7 +118,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             if (fut != null && !fut.isDone()) {
                 fut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> t) {
-                        handleMessage(nodeId, cacheMsg);
+                        cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                            @Override public void run() {
+                                handleMessage(nodeId, cacheMsg);
+                            }
+                        });
                     }
                 });
 


[09/18] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 4b367392f5ef80aa46940562658896455f45af07
Parents: a95c862 845fb12
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Aug 31 13:03:15 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Aug 31 13:03:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |   6 +-
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../processors/platform/PlatformTarget.java     |  28 ++--
 .../platform/PlatformAbstractTarget.java        |  80 +++++++----
 .../platform/cache/PlatformCache.java           | 135 ++++++++++---------
 .../platform/cache/PlatformCacheIterator.java   |   4 +-
 .../cache/affinity/PlatformAffinity.java        |  10 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../platform/cluster/PlatformClusterGroup.java  |  21 +--
 .../platform/compute/PlatformCompute.java       |  11 +-
 .../datastreamer/PlatformDataStreamer.java      |   4 +-
 .../platform/events/PlatformEvents.java         |  20 +--
 .../platform/messaging/PlatformMessaging.java   |  11 +-
 .../platform/services/PlatformServices.java     |  33 +++--
 .../transactions/PlatformTransactions.java      |   4 +-
 16 files changed, 223 insertions(+), 160 deletions(-)
----------------------------------------------------------------------



[17/18] ignite git commit: ignite-1301: added portable protocol version info to node's attributes

Posted by ak...@apache.org.
ignite-1301: added portable protocol version info to node's attributes


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

Branch: refs/heads/ignite-843
Commit: 13355806551138d29e34d5976a93234dac6d7998
Parents: f6ddf4e
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Aug 31 16:38:06 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Aug 31 16:38:06 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  4 ++
 .../ignite/internal/IgniteNodeAttributes.java   |  3 ++
 .../discovery/GridDiscoveryManager.java         | 13 ++++-
 .../marshaller/portable/PortableMarshaller.java |  3 ++
 .../GridDiscoveryManagerAttributesSelfTest.java | 50 ++++++++++++++++++++
 5 files changed, 71 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/13355806/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 489973f..4fa3564 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -71,6 +71,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.marshaller.portable.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.spi.*;
@@ -1170,6 +1171,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         add(ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
         add(ATTR_USER_NAME, System.getProperty("user.name"));
         add(ATTR_GRID_NAME, gridName);
+        add(ATTR_PORTABLE_PROTO_VER, cfg.getMarshaller() instanceof PortableMarshaller ?
+            ((PortableMarshaller)cfg.getMarshaller()).getProtocolVersion().toString() :
+            PortableMarshaller.DFLT_PORTABLE_PROTO_VER.toString());
 
         add(ATTR_PEER_CLASSLOADING, cfg.isPeerClassLoadingEnabled());
         add(ATTR_DEPLOYMENT_MODE, cfg.getDeploymentMode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/13355806/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 10b8df0..ea3eafd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -135,6 +135,9 @@ public final class IgniteNodeAttributes {
     /** Node consistent id. */
     public static final String ATTR_NODE_CONSISTENT_ID = ATTR_PREFIX + ".consistent.id";
 
+    /** Portable protocol version. */
+    public static final String ATTR_PORTABLE_PROTO_VER = ATTR_PREFIX + ".portable.proto.ver";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/13355806/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a7363af..8437468 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -906,6 +906,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         // Fetch local node attributes once.
         String locPreferIpV4 = locNode.attribute("java.net.preferIPv4Stack");
 
+        String locPortableProtoVer = locNode.attribute(ATTR_PORTABLE_PROTO_VER);
+
         Object locMode = locNode.attribute(ATTR_DEPLOYMENT_MODE);
 
         int locJvmMajVer = nodeJavaMajorVersion(locNode);
@@ -950,11 +952,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 boolean rmtP2pEnabled = n.attribute(ATTR_PEER_CLASSLOADING);
 
                 if (locP2pEnabled != rmtP2pEnabled)
-                    throw new IgniteCheckedException("Remote node has peer class loading enabled flag different from local " +
-                        "[locId8=" + U.id8(locNode.id()) + ", locPeerClassLoading=" + locP2pEnabled +
+                    throw new IgniteCheckedException("Remote node has peer class loading enabled flag different from" +
+                        " local [locId8=" + U.id8(locNode.id()) + ", locPeerClassLoading=" + locP2pEnabled +
                         ", rmtId8=" + U.id8(n.id()) + ", rmtPeerClassLoading=" + rmtP2pEnabled +
                         ", rmtAddrs=" + U.addressesAsString(n) + ']');
             }
+
+            String rmtPortableProtoVer = n.attribute(ATTR_PORTABLE_PROTO_VER);
+
+            // In order to support backward compatibility skip the check for nodes that don't have this attribute.
+            if (rmtPortableProtoVer != null && !F.eq(locPortableProtoVer, rmtPortableProtoVer))
+                throw new IgniteCheckedException("Remote node has portable protocol version different from local " +
+                    "[locVersion=" + locPortableProtoVer + ", rmtVersion=" + rmtPortableProtoVer + ']');
         }
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/13355806/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index 2cac90e..32f8b29 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -277,6 +277,9 @@ public class PortableMarshaller extends AbstractMarshaller {
      * @param protoVer Portable protocol version.
      */
     public void setProtocolVersion(PortableProtocolVersion protoVer) {
+        if (protoVer == null)
+            throw new IllegalArgumentException("Wrong portable protocol version: " + protoVer);
+
         this.protoVer = protoVer;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/13355806/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index e76c615..7e11592 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -18,12 +18,17 @@
 package org.apache.ignite.internal.managers.discovery;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 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.testframework.junits.common.*;
 
+import java.lang.reflect.*;
+import java.util.*;
+
 import static org.apache.ignite.configuration.DeploymentMode.*;
 
 /**
@@ -151,6 +156,51 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testDifferentPortableProtocolVersions() throws Exception {
+        startGridWithPortableProtocolVer("VER_99_99_99");
+
+        try {
+            startGrid(1);
+
+            fail();
+        }
+        catch (IgniteCheckedException e) {
+            if (!e.getCause().getMessage().startsWith("Remote node has portable protocol version different from local"))
+                throw e;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNullPortableProtocolVersion() throws Exception {
+        startGridWithPortableProtocolVer(null);
+
+        // Must not fail in order to preserve backward compatibility with the nodes that don't have this property yet.
+        startGrid(1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void startGridWithPortableProtocolVer(String ver) throws Exception {
+        Ignite ignite = startGrid(0);
+
+        ClusterNode clusterNode = ignite.cluster().localNode();
+
+        Field f = clusterNode.getClass().getDeclaredField("attrs");
+        f.setAccessible(true);
+
+        Map<String, Object> attrs = new HashMap<>((Map<String, Object>)f.get(clusterNode));
+
+        attrs.put(IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER, ver);
+
+        f.set(clusterNode, attrs);
+    }
+
+    /**
      * @param preferIpV4 {@code java.net.preferIPv4Stack} system property value.
      * @throws Exception If failed.
      */


[04/18] ignite git commit: IGNITE-1331: Reworked "op" method names.

Posted by ak...@apache.org.
IGNITE-1331: Reworked "op" method names.


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

Branch: refs/heads/ignite-843
Commit: 0995037cd5c6561f059089801b5fcfe1918c1b15
Parents: e6de574
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 31 11:46:36 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 31 11:46:36 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../processors/platform/PlatformTarget.java     |  28 ++--
 .../platform/PlatformAbstractTarget.java        |  80 ++++++++----
 .../platform/cache/PlatformCache.java           | 127 ++++++++++---------
 .../platform/cache/PlatformCacheIterator.java   |   4 +-
 .../cache/affinity/PlatformAffinity.java        |  10 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../platform/cluster/PlatformClusterGroup.java  |  21 +--
 .../platform/compute/PlatformCompute.java       |  11 +-
 .../datastreamer/PlatformDataStreamer.java      |   4 +-
 .../platform/events/PlatformEvents.java         |  20 +--
 .../platform/messaging/PlatformMessaging.java   |  11 +-
 .../platform/services/PlatformServices.java     |  33 +++--
 .../transactions/PlatformTransactions.java      |   4 +-
 15 files changed, 215 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index e60fbeb..7cb65d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -48,4 +48,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public void awaitStart() throws IgniteCheckedException {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget extensions() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 8c48649..0cf2df4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -51,4 +51,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @throws IgniteCheckedException If failed.
      */
     public void awaitStart() throws IgniteCheckedException;
+
+    /**
+     * Get platform extensions. Override this method to provide any additional targets and operations you need.
+     *
+     * @return Platform extensions.
+     */
+    PlatformTarget extensions();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
index b444247..6a22453 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -26,54 +26,54 @@ import org.jetbrains.annotations.*;
 @SuppressWarnings("UnusedDeclaration")
 public interface PlatformTarget {
     /**
-     * Synchronous IN operation.
+     * Operation accepting memory stream and returning long value.
      *
      * @param type Operation type.
      * @param memPtr Memory pointer.
-     * @return Value specific for the given operation otherwise.
-     * @throws Exception If failed.
+     * @return Result.
+     * @throws Exception If case of failure.
      */
-    public int inOp(int type, long memPtr) throws Exception;
+    public long inStreamOutLong(int type, long memPtr) throws Exception;
 
     /**
-     * Synchronous IN operation which returns managed object as result.
+     * Operation accepting memory stream and returning object.
      *
      * @param type Operation type.
      * @param memPtr Memory pointer.
-     * @return Managed result.
+     * @return Result.
      * @throws Exception If case of failure.
      */
-    public Object inOpObject(int type, long memPtr) throws Exception;
+    public Object inStreamOutObject(int type, long memPtr) throws Exception;
 
     /**
-     * Synchronous OUT operation.
+     * Operation returning result to memory stream.
      *
      * @param type Operation type.
      * @param memPtr Memory pointer.
      * @throws Exception In case of failure.
      */
-    public void outOp(int type, long memPtr) throws Exception;
+    public void outStream(int type, long memPtr) throws Exception;
 
     /**
-     * Synchronous IN-OUT operation.
+     * Operation accepting one memory stream and returning result to another memory stream.
      *
      * @param type Operation type.
      * @param inMemPtr Input memory pointer.
      * @param outMemPtr Output memory pointer.
      * @throws Exception In case of failure.
      */
-    public void inOutOp(int type, long inMemPtr, long outMemPtr) throws Exception;
+    public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception;
 
     /**
-     * Synchronous IN-OUT operation with optional argument.
+     * Operation accepting an object and a memory stream and returning result to another memory stream.
      *
      * @param type Operation type.
+     * @param arg Argument (optional).
      * @param inMemPtr Input memory pointer.
      * @param outMemPtr Output memory pointer.
-     * @param arg Argument (optional).
      * @throws Exception In case of failure.
      */
-    public void inOutOp(int type, long inMemPtr, long outMemPtr, @Nullable Object arg) throws Exception;
+    public void inObjectStreamOutStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception;
 
     /**
      * Start listening for the future.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 903df0a..5864a7e 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -55,7 +55,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
-    @Override public int inOp(int type, long memPtr) throws Exception {
+    @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PortableRawReaderEx reader = platformCtx.reader(mem);
 
@@ -65,7 +65,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
                 return TRUE;
             }
             else
-                return processInOp(type, reader);
+                return processInStreamOutLong(type, reader);
         }
         catch (Exception e) {
             throw convertException(e);
@@ -73,11 +73,11 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
-    @Override public Object inOpObject(int type, long memPtr) throws Exception {
+    @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PortableRawReaderEx reader = platformCtx.reader(mem);
 
-            return processInOpObject(type, reader);
+            return processInStreamOutObject(type, reader);
         }
         catch (Exception e) {
             throw convertException(e);
@@ -85,13 +85,13 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
-    @Override public void outOp(int type, long memPtr) throws Exception {
+    @Override public void outStream(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PlatformOutputStream out = mem.output();
 
             PortableRawWriterEx writer = platformCtx.writer(out);
 
-            processOutOp(type, writer);
+            processOutStream(type, writer);
 
             out.synchronize();
         }
@@ -101,12 +101,27 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
-    @Override public void inOutOp(int type, long inMemPtr, long outMemPtr) throws Exception {
-        inOutOp(type, inMemPtr, outMemPtr, null);
+    @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
+        try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
+            PortableRawReaderEx reader = platformCtx.reader(inMem);
+
+            try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
+                PlatformOutputStream out = outMem.output();
+
+                PortableRawWriterEx writer = platformCtx.writer(out);
+
+                processInStreamOutStream(type, reader, writer);
+
+                out.synchronize();
+            }
+        }
+        catch (Exception e) {
+            throw convertException(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void inOutOp(int type, long inMemPtr, long outMemPtr, Object arg) throws Exception {
+    @Override public void inObjectStreamOutStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception {
         try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
             PortableRawReaderEx reader = platformCtx.reader(inMem);
 
@@ -115,7 +130,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
                 PortableRawWriterEx writer = platformCtx.writer(out);
 
-                processInOutOp(type, reader, writer, arg);
+                processInObjectStreamOutStream(type, arg, reader, writer);
 
                 out.synchronize();
             }
@@ -156,7 +171,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * Get current future with proper exception conversions.
      *
      * @return Future.
-     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
     protected IgniteFuture currentFutureWrapped() throws IgniteCheckedException {
@@ -185,7 +200,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * When overridden in a derived class, gets future for the current operation.
      *
      * @return current future.
-     * @throws org.apache.ignite.IgniteCheckedException
+     * @throws IgniteCheckedException
      */
     protected IgniteFuture currentFuture() throws IgniteCheckedException {
         throw new IgniteCheckedException("Future listening is not supported in " + this.getClass());
@@ -207,46 +222,59 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param type Type.
      * @param reader Portable reader.
      * @return Result.
-     * @throws org.apache.ignite.IgniteCheckedException In case of exception.
+     * @throws IgniteCheckedException In case of exception.
      */
-    protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
     /**
+     * Process IN-OUT operation.
+     *
+     * @param type Type.
+     * @param reader Portable reader.
+     * @param writer Portable writer.
+     * @throws IgniteCheckedException In case of exception.
+     */
+    protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
+        throwUnsupported(type);
+    }
+
+    /**
      * Process IN operation with managed object as result.
      *
      * @param type Type.
      * @param reader Portable reader.
      * @return Result.
-     * @throws org.apache.ignite.IgniteCheckedException In case of exception.
+     * @throws IgniteCheckedException In case of exception.
      */
-    protected Object processInOpObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    protected Object processInStreamOutObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
     /**
-     * Process OUT operation.
+     * Process IN-OUT operation.
      *
      * @param type Type.
+     * @param arg Argument.
+     * @param reader Portable reader.
      * @param writer Portable writer.
-     * @throws org.apache.ignite.IgniteCheckedException In case of exception.
+     * @throws IgniteCheckedException In case of exception.
      */
-    protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    protected void processInObjectStreamOutStream(int type, @Nullable Object arg, PortableRawReaderEx reader,
+        PortableRawWriterEx writer) throws IgniteCheckedException {
         throwUnsupported(type);
     }
 
     /**
-     * Process IN-OUT operation.
+     * Process OUT operation.
      *
      * @param type Type.
-     * @param reader Portable reader.
      * @param writer Portable writer.
-     * @param arg Argument.
-     * @throws org.apache.ignite.IgniteCheckedException In case of exception.
+     * @throws IgniteCheckedException In case of exception.
      */
-    protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        @Nullable Object arg) throws IgniteCheckedException {
+    protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         throwUnsupported(type);
     }
 
@@ -255,7 +283,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      *
      * @param type Operation type.
      * @return Dummy value which is never returned.
-     * @throws org.apache.ignite.IgniteCheckedException Exception to be thrown.
+     * @throws IgniteCheckedException Exception to be thrown.
      */
     protected <T> T throwUnsupported(int type) throws IgniteCheckedException {
         throw new IgniteCheckedException("Unsupported operation type: " + type);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index dff9d67..0347d4d 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -256,7 +256,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PUT:
                 cache.put(reader.readObjectDetached(), reader.readObjectDetached());
@@ -343,7 +343,7 @@ public class PlatformCache extends PlatformAbstractTarget {
                 return cache.isLocalLocked(reader.readObjectDetached(), reader.readBoolean()) ? TRUE : FALSE;
 
             default:
-                throw new IgniteCheckedException("Unsupported operation type: " + type);
+                return super.processInStreamOutLong(type, reader);
         }
 
         return TRUE;
@@ -369,7 +369,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInOpObject(int type, PortableRawReaderEx reader)
+    @Override protected Object processInStreamOutObject(int type, PortableRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_QRY_SQL:
@@ -402,7 +402,7 @@ public class PlatformCache extends PlatformAbstractTarget {
             }
 
             default:
-                return throwUnsupported(type);
+                return super.processInStreamOutObject(type, reader);
         }
     }
 
@@ -428,81 +428,81 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, PortableRawWriterEx w) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_NAME:
-                w.writeObject(cache.getName());
+                writer.writeObject(cache.getName());
 
                 break;
 
             case OP_METRICS:
                 CacheMetrics metrics = cache.metrics();
 
-                w.writeLong(metrics.getCacheGets());
-                w.writeLong(metrics.getCachePuts());
-                w.writeLong(metrics.getCacheHits());
-                w.writeLong(metrics.getCacheMisses());
-                w.writeLong(metrics.getCacheTxCommits());
-                w.writeLong(metrics.getCacheTxRollbacks());
-                w.writeLong(metrics.getCacheEvictions());
-                w.writeLong(metrics.getCacheRemovals());
-                w.writeFloat(metrics.getAveragePutTime());
-                w.writeFloat(metrics.getAverageGetTime());
-                w.writeFloat(metrics.getAverageRemoveTime());
-                w.writeFloat(metrics.getAverageTxCommitTime());
-                w.writeFloat(metrics.getAverageTxRollbackTime());
-                w.writeString(metrics.name());
-                w.writeLong(metrics.getOverflowSize());
-                w.writeLong(metrics.getOffHeapEntriesCount());
-                w.writeLong(metrics.getOffHeapAllocatedSize());
-                w.writeInt(metrics.getSize());
-                w.writeInt(metrics.getKeySize());
-                w.writeBoolean(metrics.isEmpty());
-                w.writeInt(metrics.getDhtEvictQueueCurrentSize());
-                w.writeInt(metrics.getTxThreadMapSize());
-                w.writeInt(metrics.getTxXidMapSize());
-                w.writeInt(metrics.getTxCommitQueueSize());
-                w.writeInt(metrics.getTxPrepareQueueSize());
-                w.writeInt(metrics.getTxStartVersionCountsSize());
-                w.writeInt(metrics.getTxCommittedVersionsSize());
-                w.writeInt(metrics.getTxRolledbackVersionsSize());
-                w.writeInt(metrics.getTxDhtThreadMapSize());
-                w.writeInt(metrics.getTxDhtXidMapSize());
-                w.writeInt(metrics.getTxDhtCommitQueueSize());
-                w.writeInt(metrics.getTxDhtPrepareQueueSize());
-                w.writeInt(metrics.getTxDhtStartVersionCountsSize());
-                w.writeInt(metrics.getTxDhtCommittedVersionsSize());
-                w.writeInt(metrics.getTxDhtRolledbackVersionsSize());
-                w.writeBoolean(metrics.isWriteBehindEnabled());
-                w.writeInt(metrics.getWriteBehindFlushSize());
-                w.writeInt(metrics.getWriteBehindFlushThreadCount());
-                w.writeLong(metrics.getWriteBehindFlushFrequency());
-                w.writeInt(metrics.getWriteBehindStoreBatchSize());
-                w.writeInt(metrics.getWriteBehindTotalCriticalOverflowCount());
-                w.writeInt(metrics.getWriteBehindCriticalOverflowCount());
-                w.writeInt(metrics.getWriteBehindErrorRetryCount());
-                w.writeInt(metrics.getWriteBehindBufferSize());
-                w.writeString(metrics.getKeyType());
-                w.writeString(metrics.getValueType());
-                w.writeBoolean(metrics.isStoreByValue());
-                w.writeBoolean(metrics.isStatisticsEnabled());
-                w.writeBoolean(metrics.isManagementEnabled());
-                w.writeBoolean(metrics.isReadThrough());
-                w.writeBoolean(metrics.isWriteThrough());
-                w.writeFloat(metrics.getCacheHitPercentage());
-                w.writeFloat(metrics.getCacheMissPercentage());
+                writer.writeLong(metrics.getCacheGets());
+                writer.writeLong(metrics.getCachePuts());
+                writer.writeLong(metrics.getCacheHits());
+                writer.writeLong(metrics.getCacheMisses());
+                writer.writeLong(metrics.getCacheTxCommits());
+                writer.writeLong(metrics.getCacheTxRollbacks());
+                writer.writeLong(metrics.getCacheEvictions());
+                writer.writeLong(metrics.getCacheRemovals());
+                writer.writeFloat(metrics.getAveragePutTime());
+                writer.writeFloat(metrics.getAverageGetTime());
+                writer.writeFloat(metrics.getAverageRemoveTime());
+                writer.writeFloat(metrics.getAverageTxCommitTime());
+                writer.writeFloat(metrics.getAverageTxRollbackTime());
+                writer.writeString(metrics.name());
+                writer.writeLong(metrics.getOverflowSize());
+                writer.writeLong(metrics.getOffHeapEntriesCount());
+                writer.writeLong(metrics.getOffHeapAllocatedSize());
+                writer.writeInt(metrics.getSize());
+                writer.writeInt(metrics.getKeySize());
+                writer.writeBoolean(metrics.isEmpty());
+                writer.writeInt(metrics.getDhtEvictQueueCurrentSize());
+                writer.writeInt(metrics.getTxThreadMapSize());
+                writer.writeInt(metrics.getTxXidMapSize());
+                writer.writeInt(metrics.getTxCommitQueueSize());
+                writer.writeInt(metrics.getTxPrepareQueueSize());
+                writer.writeInt(metrics.getTxStartVersionCountsSize());
+                writer.writeInt(metrics.getTxCommittedVersionsSize());
+                writer.writeInt(metrics.getTxRolledbackVersionsSize());
+                writer.writeInt(metrics.getTxDhtThreadMapSize());
+                writer.writeInt(metrics.getTxDhtXidMapSize());
+                writer.writeInt(metrics.getTxDhtCommitQueueSize());
+                writer.writeInt(metrics.getTxDhtPrepareQueueSize());
+                writer.writeInt(metrics.getTxDhtStartVersionCountsSize());
+                writer.writeInt(metrics.getTxDhtCommittedVersionsSize());
+                writer.writeInt(metrics.getTxDhtRolledbackVersionsSize());
+                writer.writeBoolean(metrics.isWriteBehindEnabled());
+                writer.writeInt(metrics.getWriteBehindFlushSize());
+                writer.writeInt(metrics.getWriteBehindFlushThreadCount());
+                writer.writeLong(metrics.getWriteBehindFlushFrequency());
+                writer.writeInt(metrics.getWriteBehindStoreBatchSize());
+                writer.writeInt(metrics.getWriteBehindTotalCriticalOverflowCount());
+                writer.writeInt(metrics.getWriteBehindCriticalOverflowCount());
+                writer.writeInt(metrics.getWriteBehindErrorRetryCount());
+                writer.writeInt(metrics.getWriteBehindBufferSize());
+                writer.writeString(metrics.getKeyType());
+                writer.writeString(metrics.getValueType());
+                writer.writeBoolean(metrics.isStoreByValue());
+                writer.writeBoolean(metrics.isStatisticsEnabled());
+                writer.writeBoolean(metrics.isManagementEnabled());
+                writer.writeBoolean(metrics.isReadThrough());
+                writer.writeBoolean(metrics.isWriteThrough());
+                writer.writeFloat(metrics.getCacheHitPercentage());
+                writer.writeFloat(metrics.getCacheMissPercentage());
 
                 break;
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_GET: {
                 writer.writeObjectDetached(cache.get(reader.readObjectDetached()));
@@ -593,6 +593,9 @@ public class PlatformCache extends PlatformAbstractTarget {
                 writer.writeLong(registerLock(cache.lockAll(PlatformUtils.readCollection(reader))));
 
                 break;
+
+            default:
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
index 45e777d..7494fc8 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -47,7 +47,7 @@ public class PlatformCacheIterator extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_NEXT:
                 if (iter.hasNext()) {
@@ -66,7 +66,7 @@ public class PlatformCacheIterator extends PlatformAbstractTarget {
                 break;
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
index d6dfcdb..4c134be 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -110,7 +110,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PARTITION:
                 return aff.partition(reader.readObjectDetached());
@@ -155,14 +155,14 @@ public class PlatformAffinity extends PlatformAbstractTarget {
             }
 
             default:
-                return throwUnsupported(type);
+                return super.processInStreamOutLong(type, reader);
         }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_PRIMARY_PARTITIONS: {
                 UUID nodeId = reader.readObject();
@@ -280,7 +280,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
             }
 
             default:
-                throwUnsupported(type);
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
index c01da0e..c357535 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -62,7 +62,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, final PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, final PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_BATCH: {
                 assert iter != null : "iterator() has not been called";
@@ -123,7 +123,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
             }
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index 1f2a002..6aebce0 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -90,7 +90,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings("deprecation")
-    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS:
                 platformCtx.writeClusterMetrics(writer, prj.metrics());
@@ -103,14 +103,14 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ConstantConditions", "deprecation"})
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object obj) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS_FILTERED: {
                 Collection<UUID> ids = PlatformUtils.readCollection(reader);
@@ -185,22 +185,23 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
             }
 
             default:
-                throwUnsupported(type);
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PING_NODE:
                 return pingNode(reader.readUuid()) ? TRUE : FALSE;
-        }
 
-        return throwUnsupported(type);
+            default:
+                return super.processInStreamOutLong(type, reader);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInOpObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected Object processInStreamOutObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_FOR_NODE_IDS: {
                 Collection<UUID> ids = PlatformUtils.readCollection(reader);
@@ -239,7 +240,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
             }
 
             default:
-                return throwUnsupported(type);
+                return super.processInStreamOutObject(type, reader);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 2b1f6be..3a0d031 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -67,7 +67,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_UNICAST:
                 processClosures(reader.readLong(), reader, false, false);
@@ -85,7 +85,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
                 return TRUE;
 
             default:
-                return throwUnsupported(type);
+                return super.processInStreamOutLong(type, reader);
         }
     }
 
@@ -162,8 +162,8 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_EXEC:
                 writer.writeObjectDetached(executeJavaTask(reader, false));
@@ -174,8 +174,9 @@ public class PlatformCompute extends PlatformAbstractTarget {
                 writer.writeObjectDetached(executeJavaTask(reader, true));
 
                 break;
+
             default:
-                throwUnsupported(type);
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index fc9f535..f953d36 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -83,7 +83,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc}  */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_UPDATE:
                 int plc = reader.readInt();
@@ -132,7 +132,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
                 return TRUE;
 
             default:
-                return throwUnsupported(type);
+                return super.processInStreamOutLong(type, reader);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
index befc3bd..257c291 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -130,11 +130,12 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_RECORD_LOCAL:
                 // TODO: GG-10244
-                break;
+                return TRUE;
 
             case OP_ENABLE_LOCAL:
 
@@ -152,15 +153,16 @@ public class PlatformEvents extends PlatformAbstractTarget {
                 events.stopRemoteListen(reader.readUuid());
 
                 return TRUE;
-        }
 
-        throw new IgniteCheckedException("Unsupported operation type: " + type);
+            default:
+                return super.processInStreamOutLong(type, reader);
+        }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_LOCAL_QUERY: {
                 Collection<EventAdapter> result =
@@ -242,12 +244,12 @@ public class PlatformEvents extends PlatformAbstractTarget {
             }
 
             default:
-                throw new IgniteCheckedException("Unsupported operation type: " + type);
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_ENABLED_EVENTS:
                 writeEventTypes(events.enabledEvents(), writer);
@@ -255,7 +257,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
                 break;
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
index ffc2ab3..c09e149 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -81,7 +81,8 @@ public class PlatformMessaging extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_SEND:
                 messaging.send(reader.readObjectDetached(), reader.readObjectDetached());
@@ -125,14 +126,14 @@ public class PlatformMessaging extends PlatformAbstractTarget {
             }
 
             default:
-                throw new IgniteCheckedException("Unsupported operation type: " + type);
+                return super.processInStreamOutLong(type, reader);
         }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_REMOTE_LISTEN:{
                 Object nativeFilter = reader.readObjectDetached();
@@ -151,7 +152,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
             }
 
             default:
-                throw new IgniteCheckedException("Unsupported operation type: " + type);
+                super.processInStreamOutStream(type, reader, writer);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index d0956f9..36a7147 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -118,7 +118,8 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_DEPLOY: {
                 ServiceConfiguration cfg = new ServiceConfiguration();
@@ -151,14 +152,15 @@ public class PlatformServices extends PlatformAbstractTarget {
 
                 return TRUE;
             }
-        }
 
-        return super.processInOp(type, reader);
+            default:
+                return super.processInStreamOutLong(type, reader);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
-        Object arg) throws IgniteCheckedException {
+    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_SERVICES: {
                 Collection<Service> svcs = services.services(reader.readString());
@@ -179,6 +181,15 @@ public class PlatformServices extends PlatformAbstractTarget {
                 return;
             }
 
+            default:
+                super.processInStreamOutStream(type, reader, writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInObjectStreamOutStream(int type, Object arg, PortableRawReaderEx reader,
+        PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
             case OP_DOTNET_INVOKE: {
                 assert arg != null;
                 assert arg instanceof PlatformDotNetService;
@@ -207,13 +218,14 @@ public class PlatformServices extends PlatformAbstractTarget {
 
                 return;
             }
-        }
 
-        super.processInOutOp(type, reader, writer, arg);
+            default:
+                super.processInObjectStreamOutStream(type, arg, reader, writer);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_DESCRIPTORS: {
                 Collection<ServiceDescriptor> descs = services.serviceDescriptors();
@@ -240,9 +252,10 @@ public class PlatformServices extends PlatformAbstractTarget {
 
                 return;
             }
-        }
 
-        super.processOutOp(type, writer);
+            default:
+                super.processOutStream(type, writer);
+        }
     }
 
     /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0995037c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index aedc380..d717255 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -227,7 +227,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_CACHE_CONFIG_PARAMETERS:
                 TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();
@@ -249,7 +249,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
                 break;
 
             default:
-                throwUnsupported(type);
+                super.processOutStream(type, writer);
         }
     }
 }
\ No newline at end of file


[10/18] ignite git commit: Ignite-1330

Posted by ak...@apache.org.
Ignite-1330


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

Branch: refs/heads/ignite-843
Commit: 4369c203349c27e16683fc6c608a7b784c3500fb
Parents: 4b36739
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Aug 31 13:12:41 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Aug 31 13:12:41 2015 +0300

----------------------------------------------------------------------
 assembly/release-base.xml | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4369c203/assembly/release-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-base.xml b/assembly/release-base.xml
index c95966a..6870284 100644
--- a/assembly/release-base.xml
+++ b/assembly/release-base.xml
@@ -33,11 +33,6 @@
         </file>
 
         <file>
-            <source>DISCLAIMER.txt</source>
-            <outputDirectory>/</outputDirectory>
-        </file>
-
-        <file>
             <source>config/ignite-log4j.xml</source>
             <outputDirectory>/config</outputDirectory>
         </file>


[07/18] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 845fb12b303fa1b3f378ab0a58456165fbd5b575
Parents: 8a742a2 a457ab1
Author: sboikov <sb...@gridgain.com>
Authored: Mon Aug 31 12:13:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Aug 31 12:13:38 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../processors/platform/PlatformTarget.java     |  28 ++--
 .../platform/PlatformAbstractTarget.java        |  80 +++++++----
 .../platform/cache/PlatformCache.java           | 135 ++++++++++---------
 .../platform/cache/PlatformCacheIterator.java   |   4 +-
 .../cache/affinity/PlatformAffinity.java        |  10 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../platform/cluster/PlatformClusterGroup.java  |  21 +--
 .../platform/compute/PlatformCompute.java       |  11 +-
 .../datastreamer/PlatformDataStreamer.java      |   4 +-
 .../platform/events/PlatformEvents.java         |  20 +--
 .../platform/messaging/PlatformMessaging.java   |  11 +-
 .../platform/services/PlatformServices.java     |  33 +++--
 .../transactions/PlatformTransactions.java      |   4 +-
 15 files changed, 218 insertions(+), 159 deletions(-)
----------------------------------------------------------------------