You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/11/23 07:44:55 UTC

[1/8] ignite git commit: debugging slowdowns

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5 15877a8ba -> 07f5a62ec


http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 76f2fbe..6f92204 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -17,13 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
@@ -57,17 +58,14 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
 
@@ -109,7 +107,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     private boolean retval;
 
     /** Error. */
-    private AtomicReference<Throwable> err = new AtomicReference<>(null);
+    private volatile Throwable err;
 
     /** Timed out flag. */
     private volatile boolean timedOut;
@@ -129,8 +127,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     private GridNearTxLocal tx;
 
     /** Topology snapshot to operate on. */
-    private AtomicReference<AffinityTopologyVersion> topVer =
-        new AtomicReference<>();
+    private volatile AffinityTopologyVersion topVer;
 
     /** Map of current values. */
     private Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
@@ -138,9 +135,6 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     /** Trackable flag. */
     private boolean trackable = true;
 
-    /** Mutex. */
-    private final Object mux = new Object();
-
     /** Keys locked so far. */
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
     @GridToStringExclude
@@ -152,6 +146,10 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     /** Skip store flag. */
     private final boolean skipStore;
 
+    /** Mappings to proceed. */
+    @GridToStringExclude
+    private Queue<GridNearLockMapping> mappings;
+
     /**
      * @param cctx Registry.
      * @param keys Keys to lock.
@@ -206,7 +204,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             cctx.time().addTimeoutObject(timeoutObj);
         }
 
-        valMap = new ConcurrentHashMap8<>(keys.size(), 1f);
+        valMap = new ConcurrentHashMap8<>();
     }
 
     /** {@inheritDoc} */
@@ -217,10 +215,8 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     /**
      * @return Entries.
      */
-    public List<GridDistributedCacheEntry> entriesCopy() {
-        synchronized (mux) {
-            return new ArrayList<>(entries);
-        }
+    public synchronized List<GridDistributedCacheEntry> entriesCopy() {
+        return new ArrayList<>(entries);
     }
 
     /**
@@ -313,6 +309,8 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         GridNearCacheEntry entry,
         UUID dhtNodeId
     ) throws GridCacheEntryRemovedException {
+        assert Thread.holdsLock(this);
+
         // Check if lock acquisition is timed out.
         if (timedOut)
             return null;
@@ -335,9 +333,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             txEntry.cached(entry);
         }
 
-        synchronized (mux) {
-            entries.add(entry);
-        }
+        entries.add(entry);
 
         if (c == null && timeout < 0) {
             if (log.isDebugEnabled())
@@ -525,7 +521,10 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      * @param t Error.
      */
     private void onError(Throwable t) {
-        err.compareAndSet(null, t instanceof GridCacheLockTimeoutException ? null : t);
+        synchronized (this) {
+            if (err == null)
+                err = t;
+        }
     }
 
     /**
@@ -572,35 +571,39 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      */
     private boolean checkLocks() {
         if (!isDone() && initialized() && !hasPending()) {
-            for (int i = 0; i < entries.size(); i++) {
-                while (true) {
-                    GridCacheEntryEx cached = entries.get(i);
+            synchronized (this) {
+                for (int i = 0; i < entries.size(); i++) {
+                    while (true) {
+                        GridCacheEntryEx cached = entries.get(i);
 
-                    try {
-                        if (!locked(cached)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Lock is still not acquired for entry (will keep waiting) [entry=" +
-                                    cached + ", fut=" + this + ']');
+                        try {
+                            if (!locked(cached)) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Lock is still not acquired for entry (will keep waiting) [entry=" +
+                                        cached + ", fut=" + this + ']');
 
-                            return false;
-                        }
+                                return false;
+                            }
 
-                        break;
-                    }
-                    // Possible in concurrent cases, when owner is changed after locks
-                    // have been released or cancelled.
-                    catch (GridCacheEntryRemovedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry in onOwnerChanged method (will retry): " + cached);
+                            break;
+                        }
+                        // Possible in concurrent cases, when owner is changed after locks
+                        // have been released or cancelled.
+                        catch (GridCacheEntryRemovedException ignore) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got removed entry in onOwnerChanged method (will retry): " + cached);
 
-                        // Replace old entry with new one.
-                        entries.set(i, (GridDistributedCacheEntry)cctx.cache().entryEx(cached.key()));
+                            // Replace old entry with new one.
+                            entries.set(
+                                i,
+                                (GridDistributedCacheEntry)cctx.cache().entryEx(cached.key()));
+                        }
                     }
                 }
-            }
 
-            if (log.isDebugEnabled())
-                log.debug("Local lock acquired for entries [fut=" + this + ", entries=" + entries + "]");
+                if (log.isDebugEnabled())
+                    log.debug("Local lock acquired for entries [fut=" + this + ", entries=" + entries + "]");
+            }
 
             onComplete(true, true);
 
@@ -627,7 +630,8 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         if (isDone() || (err == null && success && !checkLocks()))
             return false;
 
-        this.err.compareAndSet(null, err instanceof GridCacheLockTimeoutException ? null : err);
+        if (err != null && !(err instanceof GridCacheLockTimeoutException))
+            onError(err);
 
         if (err != null)
             success = false;
@@ -653,7 +657,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         if (tx != null)
             cctx.tm().txContext(tx);
 
-        if (super.onDone(success, err.get())) {
+        if (super.onDone(success, err)) {
             if (log.isDebugEnabled())
                 log.debug("Completing future: " + this);
 
@@ -730,7 +734,8 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             }
 
             // Continue mapping on the same topology version as it was before.
-            this.topVer.compareAndSet(null, topVer);
+            if (this.topVer == null)
+                this.topVer = topVer;
 
             map(keys, false, true);
 
@@ -749,7 +754,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      *
      * @param remap Remap flag.
      */
-    void mapOnTopology(final boolean remap) {
+    synchronized void mapOnTopology(final boolean remap) {
         // We must acquire topology snapshot from the topology version future.
         cctx.topology().readLock();
 
@@ -778,13 +783,14 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
                     if (tx != null)
                         tx.onRemap(topVer);
 
-                    this.topVer.set(topVer);
+                    this.topVer = topVer;
                 }
                 else {
                     if (tx != null)
                         tx.topologyVersion(topVer);
 
-                    this.topVer.compareAndSet(null, topVer);
+                    if (this.topVer == null)
+                        this.topVer = topVer;
                 }
 
                 map(keys, remap, false);
@@ -825,7 +831,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      */
     private void map(Iterable<KeyCacheObject> keys, boolean remap, boolean topLocked) {
         try {
-            AffinityTopologyVersion topVer = this.topVer.get();
+            AffinityTopologyVersion topVer = this.topVer;
 
             assert topVer != null;
 
@@ -842,204 +848,227 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
 
             assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
 
-            ConcurrentLinkedDeque8<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
+            synchronized (this) {
+                mappings = new ArrayDeque<>();
 
-            // Assign keys to primary nodes.
-            GridNearLockMapping map = null;
+                // Assign keys to primary nodes.
+                GridNearLockMapping map = null;
 
-            for (KeyCacheObject key : keys) {
-                GridNearLockMapping updated = map(key, map, topVer);
+                for (KeyCacheObject key : keys) {
+                    GridNearLockMapping updated = map(
+                        key,
+                        map,
+                        topVer);
 
-                // If new mapping was created, add to collection.
-                if (updated != map) {
-                    mappings.add(updated);
+                    // If new mapping was created, add to collection.
+                    if (updated != map) {
+                        mappings.add(updated);
 
-                    if (tx != null && updated.node().isLocal())
-                        tx.nearLocallyMapped(true);
+                        if (tx != null && updated.node().isLocal())
+                            tx.nearLocallyMapped(true);
+                    }
+
+                    map = updated;
                 }
 
-                map = updated;
-            }
+                if (isDone()) {
+                    if (log.isDebugEnabled())
+                        log.debug("Abandoning (re)map because future is done: " + this);
 
-            if (isDone()) {
-                if (log.isDebugEnabled())
-                    log.debug("Abandoning (re)map because future is done: " + this);
+                    return;
+                }
 
-                return;
-            }
+                if (log.isDebugEnabled())
+                    log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
 
-            if (log.isDebugEnabled())
-                log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
+                boolean first = true;
 
-            boolean first = true;
+                // Create mini futures.
+                for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
+                    GridNearLockMapping mapping = iter.next();
 
-            // Create mini futures.
-            for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
-                GridNearLockMapping mapping = iter.next();
+                    ClusterNode node = mapping.node();
+                    Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
 
-                ClusterNode node = mapping.node();
-                Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
+                    assert !mappedKeys.isEmpty();
 
-                assert !mappedKeys.isEmpty();
+                    GridNearLockRequest req = null;
 
-                GridNearLockRequest req = null;
+                    Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
 
-                Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
+                    boolean explicit = false;
 
-                boolean explicit = false;
+                    for (KeyCacheObject key : mappedKeys) {
+                        IgniteTxKey txKey = cctx.txKey(key);
 
-                for (KeyCacheObject key : mappedKeys) {
-                    IgniteTxKey txKey = cctx.txKey(key);
+                        while (true) {
+                            GridNearCacheEntry entry = null;
 
-                    while (true) {
-                        GridNearCacheEntry entry = null;
+                            try {
+                                entry = cctx.near().entryExx(
+                                    key,
+                                    topVer);
 
-                        try {
-                            entry = cctx.near().entryExx(key, topVer);
+                                if (!cctx.isAll(
+                                    entry,
+                                    filter)) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Entry being locked did not pass filter (will not lock): " + entry);
 
-                            if (!cctx.isAll(entry, filter)) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Entry being locked did not pass filter (will not lock): " + entry);
+                                    onComplete(
+                                        false,
+                                        false);
 
-                                onComplete(false, false);
+                                    return;
+                                }
 
-                                return;
-                            }
+                                // Removed exception may be thrown here.
+                                GridCacheMvccCandidate cand = addEntry(
+                                    topVer,
+                                    entry,
+                                    node.id());
 
-                            // Removed exception may be thrown here.
-                            GridCacheMvccCandidate cand = addEntry(topVer, entry, node.id());
+                                if (isDone()) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Abandoning (re)map because future is done after addEntry attempt " +
+                                            "[fut=" + this + ", entry=" + entry + ']');
 
-                            if (isDone()) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Abandoning (re)map because future is done after addEntry attempt " +
-                                        "[fut=" + this + ", entry=" + entry + ']');
+                                    return;
+                                }
 
-                                return;
-                            }
+                                if (cand != null) {
+                                    if (tx == null && !cand.reentry())
+                                        cctx.mvcc().addExplicitLock(
+                                            threadId,
+                                            cand,
+                                            topVer);
 
-                            if (cand != null) {
-                                if (tx == null && !cand.reentry())
-                                    cctx.mvcc().addExplicitLock(threadId, cand, topVer);
+                                    IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.versionedValue();
 
-                                IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.versionedValue();
+                                    if (val == null) {
+                                        GridDhtCacheEntry dhtEntry = dht().peekExx(key);
 
-                                if (val == null) {
-                                    GridDhtCacheEntry dhtEntry = dht().peekExx(key);
+                                        try {
+                                            if (dhtEntry != null)
+                                                val = dhtEntry.versionedValue(topVer);
+                                        }
+                                        catch (GridCacheEntryRemovedException ignored) {
+                                            assert dhtEntry.obsolete() : dhtEntry;
 
-                                    try {
-                                        if (dhtEntry != null)
-                                            val = dhtEntry.versionedValue(topVer);
+                                            if (log.isDebugEnabled())
+                                                log.debug("Got removed exception for DHT entry in map (will ignore): "
+                                                    + dhtEntry);
+                                        }
                                     }
-                                    catch (GridCacheEntryRemovedException ignored) {
-                                        assert dhtEntry.obsolete() : " Got removed exception for non-obsolete entry: "
-                                            + dhtEntry;
 
-                                        if (log.isDebugEnabled())
-                                            log.debug("Got removed exception for DHT entry in map (will ignore): "
-                                                + dhtEntry);
-                                    }
-                                }
+                                    GridCacheVersion dhtVer = null;
 
-                                GridCacheVersion dhtVer = null;
+                                    if (val != null) {
+                                        dhtVer = val.get1();
 
-                                if (val != null) {
-                                    dhtVer = val.get1();
+                                        valMap.put(
+                                            key,
+                                            val);
+                                    }
 
-                                    valMap.put(key, val);
-                                }
+                                    if (!cand.reentry()) {
+                                        if (req == null) {
+                                            boolean clientFirst = false;
 
-                                if (!cand.reentry()) {
-                                    if (req == null) {
-                                        boolean clientFirst = false;
+                                            if (first) {
+                                                clientFirst = clientNode &&
+                                                    !topLocked &&
+                                                    (tx == null || !tx.hasRemoteLocks());
 
-                                        if (first) {
-                                            clientFirst = clientNode &&
-                                                !topLocked &&
-                                                (tx == null || !tx.hasRemoteLocks());
+                                                first = false;
+                                            }
 
-                                            first = false;
+                                            req = new GridNearLockRequest(
+                                                cctx.cacheId(),
+                                                topVer,
+                                                cctx.nodeId(),
+                                                threadId,
+                                                futId,
+                                                lockVer,
+                                                inTx(),
+                                                implicitTx(),
+                                                implicitSingleTx(),
+                                                read,
+                                                retval,
+                                                isolation(),
+                                                isInvalidate(),
+                                                timeout,
+                                                mappedKeys.size(),
+                                                inTx() ? tx.size() : mappedKeys.size(),
+                                                inTx() && tx.syncCommit(),
+                                                inTx() ? tx.subjectId() : null,
+                                                inTx() ? tx.taskNameHash() : 0,
+                                                read ? accessTtl : -1L,
+                                                skipStore,
+                                                clientFirst,
+                                                cctx.deploymentEnabled());
+
+                                            mapping.request(req);
                                         }
 
-                                        req = new GridNearLockRequest(
-                                            cctx.cacheId(),
-                                            topVer,
-                                            cctx.nodeId(),
-                                            threadId,
-                                            futId,
-                                            lockVer,
-                                            inTx(),
-                                            implicitTx(),
-                                            implicitSingleTx(),
-                                            read,
-                                            retval,
-                                            isolation(),
-                                            isInvalidate(),
-                                            timeout,
-                                            mappedKeys.size(),
-                                            inTx() ? tx.size() : mappedKeys.size(),
-                                            inTx() && tx.syncCommit(),
-                                            inTx() ? tx.subjectId() : null,
-                                            inTx() ? tx.taskNameHash() : 0,
-                                            read ? accessTtl : -1L,
-                                            skipStore,
-                                            clientFirst,
-                                            cctx.deploymentEnabled());
-
-                                        mapping.request(req);
-                                    }
+                                        distributedKeys.add(key);
 
-                                    distributedKeys.add(key);
+                                        if (tx != null)
+                                            tx.addKeyMapping(
+                                                txKey,
+                                                mapping.node());
 
-                                    if (tx != null)
-                                        tx.addKeyMapping(txKey, mapping.node());
+                                        req.addKeyBytes(
+                                            key,
+                                            retval && dhtVer == null,
+                                            dhtVer,
+                                            // Include DHT version to match remote DHT entry.
+                                            cctx);
+                                    }
 
-                                    req.addKeyBytes(
-                                        key,
-                                        retval && dhtVer == null,
-                                        dhtVer, // Include DHT version to match remote DHT entry.
-                                        cctx);
+                                    if (cand.reentry())
+                                        explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
                                 }
-
-                                if (cand.reentry())
+                                else
+                                    // Ignore reentries within transactions.
                                     explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
-                            }
-                            else
-                                // Ignore reentries within transactions.
-                                explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
 
-                            if (explicit)
-                                tx.addKeyMapping(txKey, mapping.node());
+                                if (explicit)
+                                    tx.addKeyMapping(
+                                        txKey,
+                                        mapping.node());
 
-                            break;
-                        }
-                        catch (GridCacheEntryRemovedException ignored) {
-                            assert entry.obsolete() : "Got removed exception on non-obsolete entry: " + entry;
+                                break;
+                            }
+                            catch (GridCacheEntryRemovedException ignored) {
+                                assert entry.obsolete() : "Got removed exception on non-obsolete entry: " + entry;
 
-                            if (log.isDebugEnabled())
-                                log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
+                                if (log.isDebugEnabled())
+                                    log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
+                            }
                         }
-                    }
 
-                    // Mark mapping explicit lock flag.
-                    if (explicit) {
-                        boolean marked = tx != null && tx.markExplicit(node.id());
+                        // Mark mapping explicit lock flag.
+                        if (explicit) {
+                            boolean marked = tx != null && tx.markExplicit(node.id());
 
-                        assert tx == null || marked;
+                            assert tx == null || marked;
+                        }
                     }
-                }
 
-                if (!distributedKeys.isEmpty())
-                    mapping.distributedKeys(distributedKeys);
-                else {
-                    assert mapping.request() == null;
+                    if (!distributedKeys.isEmpty())
+                        mapping.distributedKeys(distributedKeys);
+                    else {
+                        assert mapping.request() == null;
 
-                    iter.remove();
+                        iter.remove();
+                    }
                 }
             }
 
             cctx.mvcc().recheckPendingLocks();
 
-            proceedMapping(mappings);
+            proceedMapping();
         }
         catch (IgniteCheckedException ex) {
             onError(ex);
@@ -1050,13 +1079,16 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      * Gets next near lock mapping and either acquires dht locks locally or sends near lock request to
      * remote primary node.
      *
-     * @param mappings Queue of mappings.
      * @throws IgniteCheckedException If mapping can not be completed.
      */
     @SuppressWarnings("unchecked")
-    private void proceedMapping(final ConcurrentLinkedDeque8<GridNearLockMapping> mappings)
+    private void proceedMapping()
         throws IgniteCheckedException {
-        GridNearLockMapping map = mappings.poll();
+        GridNearLockMapping map;
+
+        synchronized (this) {
+            map = mappings.poll();
+        }
 
         // If there are no more mappings to process, complete the future.
         if (map == null)
@@ -1139,7 +1171,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
 
                                         // Lock is held at this point, so we can set the
                                         // returned value if any.
-                                        entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer.get());
+                                        entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
 
                                         entry.readyNearLock(lockVer, mappedVer, res.committedVersions(),
                                             res.rolledbackVersions(), res.pending());
@@ -1181,9 +1213,11 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
                                             log.debug("Failed to add candidates because entry was " +
                                                 "removed (will renew).");
 
-                                        // Replace old entry with new one.
-                                        entries.set(i, (GridDistributedCacheEntry)
-                                            cctx.cache().entryEx(entry.key()));
+                                        synchronized (GridNearLockFuture.this) {
+                                            // Replace old entry with new one.
+                                            entries.set(i,
+                                                (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
+                                        }
                                     }
                                 }
 
@@ -1191,7 +1225,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
                             }
 
                             // Proceed and add new future (if any) before completing embedded future.
-                            proceedMapping(mappings);
+                            proceedMapping();
                         }
                         catch (IgniteCheckedException ex) {
                             onError(ex);
@@ -1205,7 +1239,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
                 fut));
         }
         else {
-            final MiniFuture fut = new MiniFuture(node, mappedKeys, mappings);
+            final MiniFuture fut = new MiniFuture(node, mappedKeys);
 
             req.miniId(fut.futureId());
 
@@ -1302,7 +1336,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to acquire lock for keys " +
             "(primary node left grid, retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
 
-        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer.get()));
+        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer));
 
         return topEx;
     }
@@ -1354,23 +1388,19 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         @GridToStringInclude
         private Collection<KeyCacheObject> keys;
 
-        /** Mappings to proceed. */
-        @GridToStringExclude
-        private ConcurrentLinkedDeque8<GridNearLockMapping> mappings;
-
         /** */
-        private AtomicBoolean rcvRes = new AtomicBoolean(false);
+        private boolean rcvRes;
 
         /**
          * @param node Node.
          * @param keys Keys.
-         * @param mappings Mappings to proceed.
          */
-        MiniFuture(ClusterNode node, Collection<KeyCacheObject> keys,
-            ConcurrentLinkedDeque8<GridNearLockMapping> mappings) {
+        MiniFuture(
+            ClusterNode node,
+            Collection<KeyCacheObject> keys
+        ) {
             this.node = node;
             this.keys = keys;
-            this.mappings = mappings;
         }
 
         /**
@@ -1395,197 +1425,194 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         }
 
         /**
-         * @param e Error.
-         */
-        void onResult(Throwable e) {
-            if (rcvRes.compareAndSet(false, true)) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to get future result [fut=" + this + ", err=" + e + ']');
-
-                // Fail.
-                onDone(e);
-            }
-            else
-                U.warn(log, "Received error after another result has been processed [fut=" + GridNearLockFuture.this +
-                    ", mini=" + this + ']', e);
-        }
-
-        /**
          * @param e Node left exception.
          */
         void onResult(ClusterTopologyCheckedException e) {
             if (isDone())
                 return;
 
-            if (rcvRes.compareAndSet(false, true)) {
-                if (log.isDebugEnabled())
-                    log.debug("Remote node left grid while sending or waiting for reply (will fail): " + this);
+            synchronized (this) {
+                if (!rcvRes)
+                    rcvRes = true;
+                else
+                    return;
+            }
 
-                if (tx != null)
-                    tx.removeMapping(node.id());
+            if (log.isDebugEnabled())
+                log.debug("Remote node left grid while sending or waiting for reply (will fail): " + this);
 
-                // Primary node left the grid, so fail the future.
-                GridNearLockFuture.this.onDone(newTopologyException(e, node.id()));
+            if (tx != null)
+                tx.removeMapping(node.id());
 
-                onDone(true);
-            }
+            // Primary node left the grid, so fail the future.
+            GridNearLockFuture.this.onDone(newTopologyException(e, node.id()));
+
+            onDone(true);
         }
 
         /**
          * @param res Result callback.
          */
         void onResult(GridNearLockResponse res) {
-            if (rcvRes.compareAndSet(false, true)) {
-                if (res.error() != null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
-                            ", res=" + res + ']');
+            synchronized (this) {
+                if (!rcvRes)
+                    rcvRes = true;
+                else
+                    return;
+            }
 
-                    // Fail.
-                    if (res.error() instanceof GridCacheLockTimeoutException)
-                        onDone(false);
-                    else
-                        onDone(res.error());
+            if (res.error() != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
+                        ", res=" + res + ']');
 
-                    return;
-                }
+                // Fail.
+                if (res.error() instanceof GridCacheLockTimeoutException)
+                    onDone(false);
+                else
+                    onDone(res.error());
 
-                if (res.clientRemapVersion() != null) {
-                    assert cctx.kernalContext().clientNode();
+                return;
+            }
 
-                    IgniteInternalFuture<?> affFut =
-                        cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
+            if (res.clientRemapVersion() != null) {
+                assert cctx.kernalContext().clientNode();
 
-                    if (affFut != null && !affFut.isDone()) {
-                        affFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                            @Override public void apply(IgniteInternalFuture<?> fut) {
-                                try {
-                                    fut.get();
+                IgniteInternalFuture<?> affFut =
+                    cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
 
-                                    remap();
-                                }
-                                catch (IgniteCheckedException e) {
-                                    onDone(e);
-                                }
-                                finally {
-                                    cctx.shared().txContextReset();
-                                }
+                if (affFut != null && !affFut.isDone()) {
+                    affFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut) {
+                            try {
+                                fut.get();
+
+                                remap();
                             }
-                        });
-                    }
-                    else
-                        remap();
+                            catch (IgniteCheckedException e) {
+                                onDone(e);
+                            }
+                            finally {
+                                cctx.shared().txContextReset();
+                            }
+                        }
+                    });
                 }
-                else {
-                    int i = 0;
+                else
+                    remap();
+            }
+            else {
+                int i = 0;
 
-                    AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer.get();
+                AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer;
 
-                    for (KeyCacheObject k : keys) {
-                        while (true) {
-                            GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
+                for (KeyCacheObject k : keys) {
+                    while (true) {
+                        GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
 
-                            try {
-                                if (res.dhtVersion(i) == null) {
-                                    onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
-                                        "(will fail the lock): " + res));
+                        try {
+                            if (res.dhtVersion(i) == null) {
+                                onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
+                                    "(will fail the lock): " + res));
 
-                                    return;
-                                }
+                                return;
+                            }
 
-                                IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
+                            IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
 
-                                CacheObject oldVal = entry.rawGet();
-                                boolean hasOldVal = false;
-                                CacheObject newVal = res.value(i);
+                            CacheObject oldVal = entry.rawGet();
+                            boolean hasOldVal = false;
+                            CacheObject newVal = res.value(i);
 
-                                boolean readRecordable = false;
+                            boolean readRecordable = false;
 
-                                if (retval) {
-                                    readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
+                            if (retval) {
+                                readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
 
-                                    if (readRecordable)
-                                        hasOldVal = entry.hasValue();
-                                }
+                                if (readRecordable)
+                                    hasOldVal = entry.hasValue();
+                            }
 
-                                GridCacheVersion dhtVer = res.dhtVersion(i);
-                                GridCacheVersion mappedVer = res.mappedVersion(i);
+                            GridCacheVersion dhtVer = res.dhtVersion(i);
+                            GridCacheVersion mappedVer = res.mappedVersion(i);
 
-                                if (newVal == null) {
-                                    if (oldValTup != null) {
-                                        if (oldValTup.get1().equals(dhtVer))
-                                            newVal = oldValTup.get2();
+                            if (newVal == null) {
+                                if (oldValTup != null) {
+                                    if (oldValTup.get1().equals(dhtVer))
+                                        newVal = oldValTup.get2();
 
-                                        oldVal = oldValTup.get2();
-                                    }
+                                    oldVal = oldValTup.get2();
                                 }
+                            }
 
-                                // Lock is held at this point, so we can set the
-                                // returned value if any.
-                                entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
+                            // Lock is held at this point, so we can set the
+                            // returned value if any.
+                            entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
 
-                                if (inTx()) {
-                                    tx.hasRemoteLocks(true);
+                            if (inTx()) {
+                                tx.hasRemoteLocks(true);
 
-                                    if (implicitTx() && tx.onePhaseCommit()) {
-                                        boolean pass = res.filterResult(i);
+                                if (implicitTx() && tx.onePhaseCommit()) {
+                                    boolean pass = res.filterResult(i);
 
-                                        tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
-                                    }
+                                    tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
                                 }
+                            }
 
-                                entry.readyNearLock(lockVer,
-                                    mappedVer,
-                                    res.committedVersions(),
-                                    res.rolledbackVersions(),
-                                    res.pending());
-
-                                if (retval) {
-                                    if (readRecordable)
-                                        cctx.events().addEvent(
-                                            entry.partition(),
-                                            entry.key(),
-                                            tx,
-                                            null,
-                                            EVT_CACHE_OBJECT_READ,
-                                            newVal,
-                                            newVal != null,
-                                            oldVal,
-                                            hasOldVal,
-                                            CU.subjectId(tx, cctx.shared()),
-                                            null,
-                                            inTx() ? tx.resolveTaskName() : null);
-
-                                    if (cctx.cache().configuration().isStatisticsEnabled())
-                                        cctx.cache().metrics0().onRead(false);
-                                }
+                            entry.readyNearLock(lockVer,
+                                mappedVer,
+                                res.committedVersions(),
+                                res.rolledbackVersions(),
+                                res.pending());
+
+                            if (retval) {
+                                if (readRecordable)
+                                    cctx.events().addEvent(
+                                        entry.partition(),
+                                        entry.key(),
+                                        tx,
+                                        null,
+                                        EVT_CACHE_OBJECT_READ,
+                                        newVal,
+                                        newVal != null,
+                                        oldVal,
+                                        hasOldVal,
+                                        CU.subjectId(tx, cctx.shared()),
+                                        null,
+                                        inTx() ? tx.resolveTaskName() : null);
+
+                                if (cctx.cache().configuration().isStatisticsEnabled())
+                                    cctx.cache().metrics0().onRead(false);
+                            }
 
-                                if (log.isDebugEnabled())
-                                    log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
 
-                                break; // Inner while loop.
-                            }
-                            catch (GridCacheEntryRemovedException ignored) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to add candidates because entry was removed (will renew).");
+                            break; // Inner while loop.
+                        }
+                        catch (GridCacheEntryRemovedException ignored) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to add candidates because entry was removed (will renew).");
 
+                            synchronized (GridNearLockFuture.this) {
                                 // Replace old entry with new one.
-                                entries.set(i, (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
+                                entries.set(i,
+                                    (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
                             }
                         }
-
-                        i++;
                     }
 
-                    try {
-                        proceedMapping(mappings);
-                    }
-                    catch (IgniteCheckedException e) {
-                        onDone(e);
-                    }
+                    i++;
+                }
 
-                    onDone(true);
+                try {
+                    proceedMapping();
+                }
+                catch (IgniteCheckedException e) {
+                    onDone(e);
                 }
+
+                onDone(true);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockMapping.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockMapping.java
index b4f689c..6c8e388 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockMapping.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockMapping.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.LinkedList;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -35,7 +35,7 @@ public class GridNearLockMapping {
 
     /** Collection of mapped keys. */
     @GridToStringInclude
-    private Collection<KeyCacheObject> mappedKeys = new LinkedList<>();
+    private final Collection<KeyCacheObject> mappedKeys = new ArrayList<>();
 
     /** Near lock request. */
     @GridToStringExclude
@@ -115,4 +115,4 @@ public class GridNearLockMapping {
     public String toString() {
         return S.toString(GridNearLockMapping.class, this);
     }
-}
\ No newline at end of file
+}


[6/8] ignite git commit: Merge branch ignite-1.5 into ignite-1.5-tx-futs-opts

Posted by ag...@apache.org.
Merge branch ignite-1.5 into ignite-1.5-tx-futs-opts


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

Branch: refs/heads/ignite-1.5
Commit: 7cd4a77a39007e890e5ae47fdcfe7a24b010c20d
Parents: 1c8ff10 15877a8
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 09:36:01 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 09:36:01 2015 +0300

----------------------------------------------------------------------
 .gitignore                                      |    6 +
 assembly/release-fabric-base.xml                |   94 +
 .../config/binary/example-ignite-binary.xml     |   44 +
 examples/config/example-default.xml             |   76 +
 examples/config/example-ignite.xml              |   56 +-
 .../apache/ignite/examples/binary/Address.java  |   72 +
 .../apache/ignite/examples/binary/Employee.java |   93 +
 .../ignite/examples/binary/EmployeeKey.java     |   90 +
 .../binary/ExampleBinaryNodeStartup.java        |   36 +
 .../ignite/examples/binary/Organization.java    |   93 +
 .../examples/binary/OrganizationType.java       |   32 +
 ...ComputeClientBinaryTaskExecutionExample.java |  153 +
 .../binary/computegrid/ComputeClientTask.java   |  116 +
 .../binary/computegrid/package-info.java        |   21 +
 .../CacheClientBinaryPutGetExample.java         |  230 ++
 .../datagrid/CacheClientBinaryQueryExample.java |  330 ++
 .../examples/binary/datagrid/package-info.java  |   21 +
 .../ignite/examples/binary/package-info.java    |   21 +
 .../CacheClientPortableExampleTest.java         |   46 +
 .../ComputeClientPortableExampleTest.java       |   37 +
 .../testsuites/IgniteExamplesSelfTestSuite.java |    6 +
 .../ignite/codegen/MessageCodeGenerator.java    |   11 +-
 modules/core/pom.xml                            |   21 +
 ...processors.platform.PlatformBootstrapFactory |    2 +
 .../src/main/java/org/apache/ignite/Ignite.java |    7 +
 .../java/org/apache/ignite/IgniteBinary.java    |  357 ++
 .../java/org/apache/ignite/IgniteCache.java     |   41 +
 .../org/apache/ignite/IgniteDataStreamer.java   |   16 +
 .../org/apache/ignite/binary/BinaryField.java   |   46 +
 .../apache/ignite/binary/BinaryIdMapper.java    |   54 +
 .../binary/BinaryInvalidTypeException.java      |   58 +
 .../org/apache/ignite/binary/BinaryObject.java  |  149 +
 .../ignite/binary/BinaryObjectBuilder.java      |  135 +
 .../ignite/binary/BinaryObjectException.java    |   57 +
 .../apache/ignite/binary/BinaryRawReader.java   |  240 ++
 .../apache/ignite/binary/BinaryRawWriter.java   |  225 ++
 .../org/apache/ignite/binary/BinaryReader.java  |  290 ++
 .../apache/ignite/binary/BinarySerializer.java  |   47 +
 .../org/apache/ignite/binary/BinaryType.java    |   68 +
 .../ignite/binary/BinaryTypeConfiguration.java  |  112 +
 .../org/apache/ignite/binary/BinaryWriter.java  |  273 ++
 .../org/apache/ignite/binary/Binarylizable.java |   48 +
 .../org/apache/ignite/binary/package-info.java  |   22 +
 .../ignite/cache/CacheKeyConfiguration.java     |   92 +
 .../apache/ignite/cache/CacheTypeMetadata.java  |   67 +-
 .../org/apache/ignite/cache/QueryEntity.java    |  217 ++
 .../org/apache/ignite/cache/QueryIndex.java     |  192 +
 .../org/apache/ignite/cache/QueryIndexType.java |   38 +
 .../configuration/BinaryConfiguration.java      |  139 +
 .../configuration/CacheConfiguration.java       |  692 +++-
 .../configuration/IgniteConfiguration.java      |   75 +-
 .../configuration/PlatformConfiguration.java    |   25 +
 .../ignite/internal/GridKernalContextImpl.java  |    4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |    9 -
 .../apache/ignite/internal/IgniteKernal.java    |   39 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    6 +
 .../internal/client/GridClientCompute.java      |    2 +-
 .../impl/connection/GridClientConnection.java   |    2 +-
 .../GridClientConnectionManagerAdapter.java     |    2 +-
 .../connection/GridClientNioTcpConnection.java  |    6 +-
 .../communication/GridIoMessageFactory.java     |    4 +-
 .../swapspace/GridSwapSpaceManager.java         |   34 -
 .../portable/BinaryCachingMetadataHandler.java  |   70 +
 .../internal/portable/BinaryEnumCache.java      |   69 +
 .../internal/portable/BinaryFieldAccessor.java  |  805 ++++
 .../internal/portable/BinaryFieldImpl.java      |  116 +
 .../portable/BinaryInternalIdMapper.java        |  161 +
 .../internal/portable/BinaryMetadata.java       |  166 +
 .../portable/BinaryMetadataCollector.java       |  277 ++
 .../portable/BinaryMetadataHandler.java         |   44 +
 .../portable/BinaryNoopMetadataHandler.java     |   53 +
 .../internal/portable/BinaryObjectEx.java       |  245 ++
 .../internal/portable/BinaryObjectImpl.java     |  581 +++
 .../portable/BinaryObjectOffheapImpl.java       |  436 +++
 .../internal/portable/BinaryRawReaderEx.java    |   33 +
 .../internal/portable/BinaryRawWriterEx.java    |   60 +
 .../internal/portable/BinaryReaderExImpl.java   | 2748 +++++++++++++
 .../internal/portable/BinaryReaderHandles.java  |  108 +
 .../portable/BinaryThreadLocalContext.java      |   69 +
 .../internal/portable/BinaryTypeImpl.java       |   81 +
 .../internal/portable/BinaryWriteMode.java      |  178 +
 .../internal/portable/BinaryWriterExImpl.java   | 1817 +++++++++
 .../internal/portable/BinaryWriterHandles.java  |  101 +
 .../portable/BinaryWriterSchemaHolder.java      |  148 +
 .../portable/GridPortableMarshaller.java        |   67 +-
 .../portable/PortableClassDescriptor.java       | 1003 ++---
 .../internal/portable/PortableContext.java      |  542 +--
 .../portable/PortableMetaDataCollector.java     |  258 --
 .../portable/PortableMetaDataHandler.java       |   44 -
 .../internal/portable/PortableMetaDataImpl.java |  150 -
 .../internal/portable/PortableObjectEx.java     |  214 -
 .../internal/portable/PortableObjectImpl.java   |  391 --
 .../portable/PortableObjectOffheapImpl.java     |  243 --
 .../portable/PortablePositionReadable.java      |   47 +
 .../internal/portable/PortablePrimitives.java   |  779 +---
 .../internal/portable/PortableRawReaderEx.java  |   33 -
 .../internal/portable/PortableRawWriterEx.java  |   60 -
 .../portable/PortableReaderContext.java         |   82 -
 .../internal/portable/PortableReaderExImpl.java | 3157 ---------------
 .../internal/portable/PortableSchema.java       |  466 +++
 .../portable/PortableSchemaRegistry.java        |  172 +
 .../PortableThreadLocalMemoryAllocator.java     |  162 -
 .../ignite/internal/portable/PortableUtils.java |  651 +++-
 .../internal/portable/PortableWriterExImpl.java | 1854 ---------
 .../internal/portable/api/IgnitePortables.java  |  362 --
 .../internal/portable/api/PortableBuilder.java  |  136 -
 .../portable/api/PortableException.java         |   57 -
 .../internal/portable/api/PortableIdMapper.java |   54 -
 .../api/PortableInvalidClassException.java      |   58 -
 .../portable/api/PortableMarshalAware.java      |   48 -
 .../portable/api/PortableMarshaller.java        |  357 --
 .../internal/portable/api/PortableMetadata.java |   60 -
 .../internal/portable/api/PortableObject.java   |  152 -
 .../portable/api/PortableProtocolVersion.java   |   41 -
 .../portable/api/PortableRawReader.java         |  234 --
 .../portable/api/PortableRawWriter.java         |  219 --
 .../internal/portable/api/PortableReader.java   |  284 --
 .../portable/api/PortableSerializer.java        |   47 -
 .../portable/api/PortableTypeConfiguration.java |  195 -
 .../internal/portable/api/PortableWriter.java   |  266 --
 .../builder/BinaryObjectBuilderImpl.java        |  577 +++
 .../portable/builder/PortableBuilderEnum.java   |    8 +-
 .../portable/builder/PortableBuilderImpl.java   |  533 ---
 .../portable/builder/PortableBuilderReader.java |  214 +-
 .../PortableBuilderSerializationAware.java      |    2 +-
 .../builder/PortableBuilderSerializer.java      |   36 +-
 .../builder/PortableEnumArrayLazyValue.java     |   12 +-
 .../portable/builder/PortableLazyArrayList.java |    8 +-
 .../builder/PortableLazyLinkedList.java         |    8 +-
 .../portable/builder/PortableLazyMap.java       |    8 +-
 .../portable/builder/PortableLazyMapEntry.java  |    2 +-
 .../portable/builder/PortableLazySet.java       |    8 +-
 .../builder/PortableModifiableLazyValue.java    |    2 +-
 .../builder/PortableObjectArrayLazyValue.java   |    8 +-
 .../builder/PortablePlainLazyValue.java         |    2 +-
 .../builder/PortablePlainPortableObject.java    |   22 +-
 .../portable/builder/PortableValueWithType.java |   13 +-
 .../streams/PortableAbstractInputStream.java    |   48 +-
 .../streams/PortableAbstractOutputStream.java   |   44 +-
 .../streams/PortableHeapInputStream.java        |   34 +-
 .../streams/PortableHeapOutputStream.java       |  106 +-
 .../portable/streams/PortableInputStream.java   |   12 +-
 .../streams/PortableMemoryAllocator.java        |   67 +-
 .../streams/PortableMemoryAllocatorChunk.java   |  117 +
 .../streams/PortableOffheapInputStream.java     |   17 +-
 .../streams/PortableOffheapOutputStream.java    |   59 +-
 .../portable/streams/PortableOutputStream.java  |   94 +
 .../streams/PortableSimpleMemoryAllocator.java  |   66 -
 .../affinity/GridAffinityAssignmentCache.java   |    3 +-
 .../affinity/GridAffinityProcessor.java         |    5 +-
 .../processors/cache/CacheInvokeEntry.java      |   11 +-
 .../processors/cache/CacheLazyEntry.java        |   23 +-
 .../internal/processors/cache/CacheObject.java  |   11 +-
 .../processors/cache/CacheObjectAdapter.java    |    2 +-
 .../cache/CacheObjectByteArrayImpl.java         |    7 +-
 .../processors/cache/CacheObjectContext.java    |  128 +-
 .../processors/cache/CacheObjectImpl.java       |    5 +
 .../processors/cache/CacheOperationContext.java |   22 +-
 .../processors/cache/GridCacheAdapter.java      |   98 +-
 .../cache/GridCacheConcurrentMap.java           |   10 +-
 .../processors/cache/GridCacheContext.java      |   14 +-
 .../processors/cache/GridCacheEntryEx.java      |   15 +-
 .../processors/cache/GridCacheEventManager.java |   31 +-
 .../cache/GridCacheEvictionManager.java         |    5 +-
 .../processors/cache/GridCacheMapEntry.java     |  157 +-
 .../processors/cache/GridCacheMvccManager.java  |   29 +-
 .../processors/cache/GridCacheProcessor.java    |    7 +
 .../processors/cache/GridCacheProxyImpl.java    |    4 +-
 .../processors/cache/GridCacheReturn.java       |   27 +-
 .../processors/cache/GridCacheSwapManager.java  |   37 +-
 .../processors/cache/GridCacheUtils.java        |    3 +
 .../processors/cache/IgniteCacheProxy.java      |   11 +-
 .../processors/cache/IgniteInternalCache.java   |    6 +-
 .../processors/cache/KeyCacheObjectImpl.java    |    5 +
 .../CacheDataStructuresManager.java             |    4 +-
 .../GridDistributedCacheAdapter.java            |   40 +-
 .../distributed/GridDistributedCacheEntry.java  |    9 +-
 .../distributed/GridDistributedLockRequest.java |   17 +
 .../GridDistributedTxPrepareRequest.java        |    3 +-
 .../GridDistributedTxRemoteAdapter.java         |    3 +
 .../distributed/dht/GridDhtCacheEntry.java      |    5 +-
 .../distributed/dht/GridDhtLocalPartition.java  |    3 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../distributed/dht/GridDhtLockRequest.java     |    3 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   21 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   17 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   15 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   12 +-
 .../dht/GridPartitionedGetFuture.java           |    6 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   14 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   74 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    6 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   42 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   15 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   68 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |    5 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   31 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   41 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    2 +-
 .../dht/preloader/GridDhtPartitionDemander.java |    2 +-
 .../distributed/near/GridNearAtomicCache.java   |    4 +
 .../distributed/near/GridNearCacheEntry.java    |   17 +-
 .../distributed/near/GridNearGetFuture.java     |   55 +-
 .../distributed/near/GridNearLockFuture.java    |   14 +-
 .../distributed/near/GridNearLockRequest.java   |    2 +
 .../near/GridNearTransactionalCache.java        |    8 +-
 .../cache/distributed/near/GridNearTxLocal.java |    9 +-
 .../distributed/near/GridNearTxRemote.java      |   12 +-
 .../cache/local/GridLocalCacheEntry.java        |   16 +-
 .../cache/local/GridLocalLockFuture.java        |    3 +-
 .../local/atomic/GridLocalAtomicCache.java      |  108 +-
 .../CacheDefaultPortableAffinityKeyMapper.java  |    8 +-
 .../portable/CacheObjectBinaryProcessor.java    |   97 +
 .../CacheObjectBinaryProcessorImpl.java         |  848 ++++
 .../portable/CacheObjectPortableContext.java    |  133 -
 .../portable/CacheObjectPortableProcessor.java  |  103 -
 .../CacheObjectPortableProcessorImpl.java       | 1035 -----
 .../cache/portable/IgniteBinaryImpl.java        |  165 +
 .../cache/portable/IgnitePortablesImpl.java     |  177 -
 .../cache/portable/PortableMetaDataKey.java     |   82 -
 .../cache/portable/PortableMetadataKey.java     |   82 +
 .../cache/query/GridCacheQueryManager.java      |  298 +-
 .../cache/query/GridCacheQueryRequest.java      |    4 +-
 .../continuous/CacheContinuousQueryEvent.java   |   10 +-
 .../cache/store/CacheOsStoreManager.java        |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   47 +-
 .../transactions/IgniteTransactionsImpl.java    |   10 +-
 .../cache/transactions/IgniteTxAdapter.java     |    5 +-
 .../cache/transactions/IgniteTxEntry.java       |   57 +-
 .../transactions/IgniteTxLocalAdapter.java      |  145 +-
 .../cache/transactions/IgniteTxLocalEx.java     |    1 +
 .../cache/transactions/IgniteTxManager.java     |    2 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    6 +
 .../IgniteCacheObjectProcessorImpl.java         |    9 +
 .../processors/cacheobject/NoOpBinary.java      |   75 +
 .../continuous/GridContinuousProcessor.java     |    6 +-
 .../datastreamer/DataStreamProcessor.java       |    1 +
 .../datastreamer/DataStreamerEntry.java         |    6 +-
 .../datastreamer/DataStreamerImpl.java          |   16 +-
 .../datastreamer/DataStreamerRequest.java       |   58 +-
 .../datastreamer/DataStreamerUpdateJob.java     |   10 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   26 +-
 .../processors/hadoop/HadoopJobProperty.java    |    2 +-
 .../platform/PlatformAbstractBootstrap.java     |   48 +
 .../PlatformAbstractConfigurationClosure.java   |   61 +
 .../platform/PlatformAbstractPredicate.java     |   67 +
 .../platform/PlatformAbstractTarget.java        |  320 ++
 .../processors/platform/PlatformBootstrap.java  |   35 +
 .../platform/PlatformBootstrapFactory.java      |   37 +
 .../platform/PlatformConfiguration.java         |   25 -
 .../platform/PlatformConfigurationEx.java       |   48 +
 .../processors/platform/PlatformContext.java    |   26 +-
 .../platform/PlatformContextImpl.java           |  616 +++
 .../platform/PlatformExtendedException.java     |    4 +-
 .../processors/platform/PlatformIgnition.java   |  189 +
 .../platform/PlatformNoopProcessor.java         |    6 +
 .../processors/platform/PlatformProcessor.java  |   11 +
 .../platform/PlatformProcessorImpl.java         |  374 ++
 .../platform/cache/PlatformCache.java           | 1090 ++++++
 .../cache/PlatformCacheEntryFilterImpl.java     |  106 +
 .../cache/PlatformCacheEntryProcessorImpl.java  |  220 ++
 .../platform/cache/PlatformCacheIterator.java   |   72 +
 .../PlatformCachePartialUpdateException.java    |   59 +
 .../cache/affinity/PlatformAffinity.java        |  297 ++
 .../query/PlatformAbstractQueryCursor.java      |  192 +
 .../query/PlatformContinuousQueryImpl.java      |  235 ++
 .../PlatformContinuousQueryRemoteFilter.java    |  188 +
 .../cache/query/PlatformFieldsQueryCursor.java  |   49 +
 .../cache/query/PlatformQueryCursor.java        |   45 +
 .../cache/store/PlatformCacheStoreCallback.java |   61 +
 .../platform/cluster/PlatformClusterGroup.java  |  335 ++
 .../cluster/PlatformClusterNodeFilterImpl.java  |   78 +
 .../platform/compute/PlatformAbstractJob.java   |  156 +
 .../platform/compute/PlatformAbstractTask.java  |  206 +
 .../PlatformBalancingMultiClosureTask.java      |   83 +
 ...tformBalancingSingleClosureAffinityTask.java |   88 +
 .../PlatformBalancingSingleClosureTask.java     |   81 +
 .../PlatformBroadcastingMultiClosureTask.java   |   87 +
 .../PlatformBroadcastingSingleClosureTask.java  |   84 +
 .../platform/compute/PlatformClosureJob.java    |  104 +
 .../platform/compute/PlatformCompute.java       |  332 ++
 .../platform/compute/PlatformFullJob.java       |  220 ++
 .../platform/compute/PlatformFullTask.java      |  192 +
 .../platform/cpp/PlatformCppBootstrap.java      |   31 +
 .../cpp/PlatformCppBootstrapFactory.java        |   39 +
 .../cpp/PlatformCppConfigurationClosure.java    |  114 +
 .../cpp/PlatformCppConfigurationEx.java         |   82 +
 .../datastreamer/PlatformDataStreamer.java      |  227 ++
 .../PlatformStreamReceiverImpl.java             |  119 +
 .../datastructures/PlatformAtomicLong.java      |  149 +
 .../dotnet/PlatformDotNetBootstrap.java         |   31 +
 .../dotnet/PlatformDotNetBootstrapFactory.java  |   39 +
 .../dotnet/PlatformDotNetCacheStore.java        |  484 +++
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetConfigurationClosure.java     |  258 ++
 .../dotnet/PlatformDotNetConfigurationEx.java   |   91 +
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../platform/dotnet/PlatformDotNetService.java  |   27 +
 .../dotnet/PlatformDotNetServiceImpl.java       |   47 +
 .../events/PlatformEventFilterListenerImpl.java |  186 +
 .../platform/events/PlatformEvents.java         |  396 ++
 .../lifecycle/PlatformLifecycleBean.java        |   75 +
 .../platform/memory/PlatformAbstractMemory.java |  121 +
 .../PlatformBigEndianInputStreamImpl.java       |  136 +
 .../PlatformBigEndianOutputStreamImpl.java      |  196 +
 .../platform/memory/PlatformExternalMemory.java |   55 +
 .../memory/PlatformInputStreamImpl.java         |  351 ++
 .../memory/PlatformMemoryManagerImpl.java       |   85 +
 .../platform/memory/PlatformMemoryPool.java     |  140 +
 .../platform/memory/PlatformMemoryUtils.java    |  467 +++
 .../memory/PlatformOutputStreamImpl.java        |  342 ++
 .../platform/memory/PlatformPooledMemory.java   |   64 +
 .../platform/memory/PlatformUnpooledMemory.java |   51 +
 .../messaging/PlatformMessageFilterImpl.java    |  110 +
 .../messaging/PlatformMessageLocalFilter.java   |  102 +
 .../platform/messaging/PlatformMessaging.java   |  166 +
 .../services/PlatformAbstractService.java       |  230 ++
 .../platform/services/PlatformServices.java     |  275 ++
 .../transactions/PlatformTransactions.java      |  259 ++
 .../platform/utils/PlatformFutureUtils.java     |  397 ++
 .../platform/utils/PlatformReaderBiClosure.java |   34 +
 .../platform/utils/PlatformReaderClosure.java   |   34 +
 .../platform/utils/PlatformUtils.java           |  809 ++++
 .../platform/utils/PlatformWriterBiClosure.java |   34 +
 .../platform/utils/PlatformWriterClosure.java   |   33 +
 .../processors/query/GridQueryIndexing.java     |   18 +-
 .../processors/query/GridQueryProcessor.java    |  674 ++--
 .../client/message/GridClientTaskRequest.java   |    6 +-
 .../ignite/internal/util/GridEnumCache.java     |   56 -
 .../ignite/internal/util/IgniteUtils.java       |   34 +-
 .../ignite/internal/util/nio/GridNioServer.java |    4 +-
 .../marshaller/portable/BinaryMarshaller.java   |  146 +
 .../marshaller/portable/package-info.java       |   22 +
 .../platform/cpp/PlatformCppConfiguration.java  |   47 +
 .../ignite/platform/cpp/package-info.java       |   22 +
 .../PlatformDotNetBinaryConfiguration.java      |  170 +
 .../PlatformDotNetBinaryTypeConfiguration.java  |  171 +
 .../dotnet/PlatformDotNetCacheStoreFactory.java |  117 +
 .../dotnet/PlatformDotNetConfiguration.java     |   97 +
 .../dotnet/PlatformDotNetLifecycleBean.java     |   86 +
 .../ignite/platform/dotnet/package-info.java    |   22 +
 .../apache/ignite/platform/package-info.java    |   22 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   51 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +
 .../apache/ignite/spi/swapspace/SwapKey.java    |    9 +-
 .../resources/META-INF/classnames.properties    |  268 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |  719 ++++
 .../portable/BinaryFieldsHeapSelfTest.java      |   32 +
 .../portable/BinaryFieldsOffheapSelfTest.java   |   61 +
 .../BinaryFooterOffsetsAbstractSelfTest.java    |  206 +
 .../BinaryFooterOffsetsHeapSelfTest.java        |   32 +
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   61 +
 .../portable/BinaryMarshallerSelfTest.java      | 3671 ++++++++++++++++++
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 1292 ++++++
 .../portable/BinaryObjectBuilderSelfTest.java   | 1069 +++++
 .../GridPortableAffinityKeySelfTest.java        |  225 ++
 ...idPortableMarshallerCtxDisabledSelfTest.java |  248 ++
 .../portable/GridPortableMetaDataSelfTest.java  |  372 ++
 .../portable/GridPortableWildcardsSelfTest.java |  465 +++
 .../portable/TestCachingMetadataHandler.java    |   45 +
 .../GridBinaryMarshalerAwareTestClass.java      |   67 +
 .../mutabletest/GridPortableTestClasses.java    |  443 +++
 .../portable/mutabletest/package-info.java      |   22 +
 .../BinaryFieldsHeapNonCompactSelfTest.java     |   30 +
 .../BinaryFieldsOffheapNonCompactSelfTest.java  |   30 +
 ...naryFooterOffsetsHeapNonCompactSelfTest.java |   30 +
 ...yFooterOffsetsOffheapNonCompactSelfTest.java |   30 +
 .../BinaryMarshallerNonCompactSelfTest.java     |   30 +
 ...jectBuilderAdditionalNonCompactSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 +
 .../ignite/internal/portable/package-info.java  |   22 +
 .../portable/test/GridPortableTestClass1.java   |   28 +
 .../portable/test/GridPortableTestClass2.java   |   24 +
 .../internal/portable/test/package-info.java    |   22 +
 .../test/subpackage/GridPortableTestClass3.java |   24 +
 .../portable/test/subpackage/package-info.java  |   22 +
 .../cache/CacheNearReaderUpdateTest.java        |   62 +-
 .../cache/GridCacheAbstractSelfTest.java        |    9 +-
 ...heOffHeapTieredEvictionAbstractSelfTest.java |    2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   13 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   30 +-
 ...IgniteCacheAbstractExecutionContextTest.java |    5 +
 ...tractDistributedByteArrayValuesSelfTest.java |    4 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |  192 +
 ...aryObjectsAbstractMultiThreadedSelfTest.java |  233 ++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  981 +++++
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |  295 ++
 ...CacheClientNodeBinaryObjectMetadataTest.java |  221 ++
 .../GridCachePortableStoreAbstractSelfTest.java |  300 ++
 .../GridCachePortableStoreObjectsSelfTest.java  |   55 +
 ...GridCachePortableStorePortablesSelfTest.java |   66 +
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   48 +
 ...leDuplicateIndexObjectsAbstractSelfTest.java |  161 +
 .../DataStreamProcessorPortableSelfTest.java    |   71 +
 .../GridDataStreamerImplSelfTest.java           |  345 ++
 ...ridCacheAffinityRoutingPortableSelfTest.java |   54 +
 ...lyPortableDataStreamerMultiNodeSelfTest.java |   29 +
 ...rtableDataStreamerMultithreadedSelfTest.java |   47 +
 ...artitionedOnlyPortableMultiNodeSelfTest.java |   28 +
 ...tionedOnlyPortableMultithreadedSelfTest.java |   47 +
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 +
 ...BinaryObjectsAtomicNearDisabledSelfTest.java |   51 +
 ...inaryObjectsAtomicOffheapTieredSelfTest.java |   29 +
 .../GridCacheBinaryObjectsAtomicSelfTest.java   |   51 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 +
 ...yObjectsPartitionedNearDisabledSelfTest.java |   51 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 +
 ...idCacheBinaryObjectsPartitionedSelfTest.java |   51 +
 .../GridCacheMemoryModePortableSelfTest.java    |   36 +
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   48 +
 ...eapTieredEvictionAtomicPortableSelfTest.java |   96 +
 ...heOffHeapTieredEvictionPortableSelfTest.java |   96 +
 .../GridCacheOffHeapTieredPortableSelfTest.java |   48 +
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 +
 ...xObjectPartitionedTransactionalSelfTest.java |   41 +
 ...sNearPartitionedByteArrayValuesSelfTest.java |   41 +
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 +
 ...ridCacheBinaryObjectsReplicatedSelfTest.java |   51 +
 ...idCacheBinaryObjectsAtomicLocalSelfTest.java |   32 +
 ...BinaryObjectsLocalOffheapTieredSelfTest.java |   29 +
 .../GridCacheBinaryObjectsLocalSelfTest.java    |   51 +
 .../CacheVersionedEntryAbstractTest.java        |    2 +-
 .../continuous/GridEventConsumeSelfTest.java    |  111 +-
 .../DataStreamProcessorSelfTest.java            |    8 +
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    3 +-
 .../platform/PlatformComputeBinarizable.java    |   42 +
 .../PlatformComputeBinarizableArgTask.java      |  119 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../PlatformComputeJavaBinarizable.java         |   39 +
 .../platform/PlatformEventsWriteEventTask.java  |  144 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 .../file/GridFileSwapSpaceSpiSelfTest.java      |   11 +-
 .../testframework/junits/GridAbstractTest.java  |    4 +-
 .../ignite/testframework/junits/IgniteMock.java |    6 +
 .../multijvm/IgniteCacheProcessProxy.java       |    7 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    4 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   37 +
 .../IgnitePortableCacheTestSuite.java           |  103 +
 .../IgnitePortableObjectsTestSuite.java         |  110 +
 .../ignite/portable/test1/1.1/test1-1.1.jar     |  Bin 0 -> 2548 bytes
 .../ignite/portable/test1/1.1/test1-1.1.pom     |    9 +
 .../portable/test1/maven-metadata-local.xml     |   12 +
 .../ignite/portable/test2/1.1/test2-1.1.jar     |  Bin 0 -> 1361 bytes
 .../ignite/portable/test2/1.1/test2-1.1.pom     |    9 +
 .../portable/test2/maven-metadata-local.xml     |   12 +
 .../HadoopDefaultMapReducePlannerSelfTest.java  |    6 -
 .../processors/query/h2/IgniteH2Indexing.java   |   25 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |    8 +-
 .../query/h2/opt/GridLuceneIndex.java           |    4 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |    3 +-
 ...CacheOffheapTieredMultithreadedSelfTest.java |   25 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   22 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |   22 +-
 .../query/IgniteSqlSplitterSelfTest.java        |   54 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |    7 +-
 .../IgnitePortableCacheQueryTestSuite.java      |  117 +
 modules/platform/licenses/apache-2.0.txt        |  202 -
 modules/platform/pom.xml                        |   80 -
 modules/platform/src/main/cpp/README.txt        |  103 -
 .../platform/src/main/cpp/common/Makefile.am    |   45 -
 .../platform/src/main/cpp/common/configure.ac   |   62 -
 .../src/main/cpp/common/ignite-common.pc.in     |    9 -
 .../src/main/cpp/common/include/Makefile.am     |   22 -
 .../common/include/ignite/common/concurrent.h   |  210 -
 .../cpp/common/include/ignite/common/exports.h  |  145 -
 .../cpp/common/include/ignite/common/java.h     |  652 ----
 .../cpp/common/os/linux/include/Makefile.am     |   21 -
 .../os/linux/include/ignite/common/common.h     |   81 -
 .../linux/include/ignite/common/concurrent_os.h |  394 --
 .../src/main/cpp/common/os/linux/src/common.cpp |   59 -
 .../cpp/common/os/linux/src/concurrent_os.cpp   |  175 -
 .../os/win/include/ignite/common/common.h       |   56 -
 .../win/include/ignite/common/concurrent_os.h   |  406 --
 .../src/main/cpp/common/os/win/src/common.cpp   |   65 -
 .../cpp/common/os/win/src/concurrent_os.cpp     |  151 -
 .../src/main/cpp/common/project/README.TXT      |    1 -
 .../src/main/cpp/common/project/vs/README.TXT   |    1 -
 .../main/cpp/common/project/vs/common.vcxproj   |  202 -
 .../common/project/vs/common.vcxproj.filters    |   54 -
 .../src/main/cpp/common/project/vs/module.def   |   99 -
 .../src/main/cpp/common/project/vs/targetver.h  |   25 -
 .../src/main/cpp/common/src/concurrent.cpp      |   94 -
 .../src/main/cpp/common/src/exports.cpp         |  413 --
 .../platform/src/main/cpp/common/src/java.cpp   | 2205 -----------
 .../platform/src/main/cpp/core-test/Makefile.am |   49 -
 .../main/cpp/core-test/config/cache-query.xml   |   91 -
 .../main/cpp/core-test/config/cache-test.xml    |  129 -
 .../src/main/cpp/core-test/configure.ac         |   62 -
 .../src/main/cpp/core-test/include/Makefile.am  |   22 -
 .../include/ignite/portable_test_defs.h         |  320 --
 .../include/ignite/portable_test_utils.h        |  516 ---
 .../cpp/core-test/include/teamcity_messages.h   |   55 -
 .../src/main/cpp/core-test/project/README.TXT   |    1 -
 .../main/cpp/core-test/project/vs/README.TXT    |    1 -
 .../cpp/core-test/project/vs/core-test.vcxproj  |  174 -
 .../project/vs/core-test.vcxproj.filters        |   68 -
 .../main/cpp/core-test/src/cache_query_test.cpp |  656 ----
 .../src/main/cpp/core-test/src/cache_test.cpp   |  486 ---
 .../main/cpp/core-test/src/concurrent_test.cpp  |  186 -
 .../cpp/core-test/src/handle_registry_test.cpp  |  176 -
 .../main/cpp/core-test/src/ignition_test.cpp    |  102 -
 .../src/portable_reader_writer_raw_test.cpp     | 1532 --------
 .../src/portable_reader_writer_test.cpp         | 1951 ----------
 .../cpp/core-test/src/portable_session_test.cpp |  257 --
 .../cpp/core-test/src/portable_test_defs.cpp    |   65 -
 .../main/cpp/core-test/src/teamcity_boost.cpp   |  159 -
 .../cpp/core-test/src/teamcity_messages.cpp     |  150 -
 modules/platform/src/main/cpp/core/Makefile.am  |   66 -
 modules/platform/src/main/cpp/core/configure.ac |   62 -
 modules/platform/src/main/cpp/core/ignite.pc.in |    9 -
 .../src/main/cpp/core/include/Makefile.am       |   61 -
 .../main/cpp/core/include/ignite/cache/cache.h  | 1153 ------
 .../cpp/core/include/ignite/cache/cache_entry.h |  118 -
 .../core/include/ignite/cache/cache_peek_mode.h |   71 -
 .../cpp/core/include/ignite/cache/query/query.h |   27 -
 .../include/ignite/cache/query/query_argument.h |  125 -
 .../include/ignite/cache/query/query_cursor.h   |  191 -
 .../include/ignite/cache/query/query_scan.h     |  151 -
 .../core/include/ignite/cache/query/query_sql.h |  253 --
 .../include/ignite/cache/query/query_text.h     |  159 -
 .../src/main/cpp/core/include/ignite/guid.h     |  112 -
 .../src/main/cpp/core/include/ignite/ignite.h   |  154 -
 .../core/include/ignite/ignite_configuration.h  |   92 -
 .../main/cpp/core/include/ignite/ignite_error.h |  260 --
 .../src/main/cpp/core/include/ignite/ignition.h |  195 -
 .../core/include/ignite/impl/cache/cache_impl.h |  418 --
 .../ignite/impl/cache/query/query_impl.h        |  115 -
 .../core/include/ignite/impl/handle_registry.h  |  202 -
 .../include/ignite/impl/ignite_environment.h    |  130 -
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  146 -
 .../core/include/ignite/impl/interop/interop.h  |   25 -
 .../ignite/impl/interop/interop_input_stream.h  |  234 --
 .../ignite/impl/interop/interop_memory.h        |  280 --
 .../ignite/impl/interop/interop_output_stream.h |  234 --
 .../cpp/core/include/ignite/impl/operations.h   |  452 ---
 .../ignite/impl/portable/portable_common.h      |  146 -
 .../ignite/impl/portable/portable_id_resolver.h |  106 -
 .../impl/portable/portable_metadata_handler.h   |  102 -
 .../impl/portable/portable_metadata_manager.h   |  120 -
 .../impl/portable/portable_metadata_snapshot.h  |  122 -
 .../impl/portable/portable_metadata_updater.h   |   53 -
 .../portable/portable_metadata_updater_impl.h   |   65 -
 .../ignite/impl/portable/portable_reader_impl.h | 1130 ------
 .../ignite/impl/portable/portable_utils.h       |  344 --
 .../ignite/impl/portable/portable_writer_impl.h |  859 ----
 .../cpp/core/include/ignite/portable/portable.h |   29 -
 .../include/ignite/portable/portable_consts.h   |  106 -
 .../ignite/portable/portable_containers.h       |  525 ---
 .../ignite/portable/portable_raw_reader.h       |  324 --
 .../ignite/portable/portable_raw_writer.h       |  300 --
 .../include/ignite/portable/portable_reader.h   |  355 --
 .../include/ignite/portable/portable_type.h     |  293 --
 .../include/ignite/portable/portable_writer.h   |  335 --
 .../main/cpp/core/os/linux/include/Makefile.am  |   20 -
 .../core/os/linux/include/ignite/impl/utils.h   |  155 -
 .../main/cpp/core/os/linux/src/impl/utils.cpp   |  439 ---
 .../cpp/core/os/win/include/ignite/impl/utils.h |  155 -
 .../src/main/cpp/core/os/win/src/impl/utils.cpp |  453 ---
 .../src/main/cpp/core/project/README.TXT        |    1 -
 .../src/main/cpp/core/project/vs/README.TXT     |    1 -
 .../src/main/cpp/core/project/vs/core.vcxproj   |  272 --
 .../cpp/core/project/vs/core.vcxproj.filters    |  246 --
 modules/platform/src/main/cpp/core/src/guid.cpp |   65 -
 .../platform/src/main/cpp/core/src/ignite.cpp   |   43 -
 .../src/main/cpp/core/src/ignite_error.cpp      |  222 --
 .../platform/src/main/cpp/core/src/ignition.cpp |  468 ---
 .../main/cpp/core/src/impl/cache/cache_impl.cpp |  388 --
 .../core/src/impl/cache/query/query_impl.cpp    |  193 -
 .../main/cpp/core/src/impl/handle_registry.cpp  |  234 --
 .../cpp/core/src/impl/ignite_environment.cpp    |  167 -
 .../src/main/cpp/core/src/impl/ignite_impl.cpp  |   42 -
 .../src/impl/interop/interop_input_stream.cpp   |  215 -
 .../core/src/impl/interop/interop_memory.cpp    |  182 -
 .../src/impl/interop/interop_output_stream.cpp  |  215 -
 .../impl/portable/portable_metadata_handler.cpp |   78 -
 .../impl/portable/portable_metadata_manager.cpp |  201 -
 .../portable/portable_metadata_snapshot.cpp     |   70 -
 .../impl/portable/portable_metadata_updater.cpp |   32 -
 .../portable/portable_metadata_updater_impl.cpp |   94 -
 .../src/impl/portable/portable_reader_impl.cpp  |  683 ----
 .../core/src/impl/portable/portable_utils.cpp   |  214 -
 .../src/impl/portable/portable_writer_impl.cpp  |  600 ---
 .../core/src/portable/portable_containers.cpp   |   76 -
 .../core/src/portable/portable_raw_reader.cpp   |  135 -
 .../core/src/portable/portable_raw_writer.cpp   |  147 -
 .../cpp/core/src/portable/portable_reader.cpp   |  142 -
 .../cpp/core/src/portable/portable_type.cpp     |   51 -
 .../cpp/core/src/portable/portable_writer.cpp   |  154 -
 .../platform/src/main/cpp/project/vs/ignite.sln |   48 -
 .../Apache.Ignite.Core.csproj                   |  376 --
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 -
 .../Cache/CacheEntryProcessorException.cs       |   79 -
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 -
 .../Cache/CachePartialUpdateException.cs        |  119 -
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 -
 .../Cache/Event/CacheEntryEventType.cs          |   41 -
 .../Cache/Event/ICacheEntryEvent.cs             |   40 -
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 -
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 -
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 -
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 -
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 ---
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 -
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 -
 .../Cache/ICacheEntryFilter.cs                  |   34 -
 .../Cache/ICacheEntryProcessor.cs               |   45 -
 .../Cache/ICacheEntryProcessorResult.cs         |   40 -
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 -
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 ---
 .../Cache/IMutableCacheEntry.cs                 |   47 -
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 -
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 -
 .../Cache/Query/IQueryCursor.cs                 |   40 -
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 -
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 -
 .../Cache/Query/SqlFieldsQuery.cs               |   81 -
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 -
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 -
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 -
 .../Cache/Store/CacheStoreAdapter.cs            |  146 -
 .../Cache/Store/CacheStoreException.cs          |   66 -
 .../Cache/Store/ICacheStore.cs                  |  184 -
 .../Cache/Store/ICacheStoreSession.cs           |   42 -
 .../Cluster/ClusterGroupEmptyException.cs       |   70 -
 .../Cluster/ClusterTopologyException.cs         |   69 -
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 -
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 --
 .../Cluster/IClusterMetrics.cs                  |  515 ---
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 -
 .../Cluster/IClusterNodeFilter.cs               |   32 -
 .../Common/AsyncSupportedAttribute.cs           |   33 -
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |   52 -
 .../dotnet/Apache.Ignite.Core/Common/IFuture.cs |  115 -
 .../Common/IgniteException.cs                   |   66 -
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 -
 .../ComputeExecutionRejectedException.cs        |   69 -
 .../Compute/ComputeJobAdapter.cs                |  122 -
 .../Compute/ComputeJobFailoverException.cs      |   72 -
 .../Compute/ComputeJobResultPolicy.cs           |   45 -
 .../Compute/ComputeTaskAdapter.cs               |   93 -
 .../Compute/ComputeTaskCancelledException.cs    |   69 -
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 -
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 -
 .../Compute/ComputeTaskTimeoutException.cs      |   67 -
 .../Compute/ComputeUserUndeclaredException.cs   |   70 -
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 --
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 -
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 -
 .../Compute/IComputeJobResult.cs                |   73 -
 .../Compute/IComputeReducer.cs                  |   39 -
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 -
 .../Datastream/IDataStreamer.cs                 |  206 -
 .../Datastream/IStreamReceiver.cs               |   38 -
 .../Datastream/StreamTransformer.cs             |   73 -
 .../Datastream/StreamVisitor.cs                 |   55 -
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 -
 .../Events/CacheQueryExecutedEvent.cs           |   97 -
 .../Events/CacheQueryReadEvent.cs               |  134 -
 .../Events/CacheRebalancingEvent.cs             |   98 -
 .../Events/CheckpointEvent.cs                   |   50 -
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 -
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 -
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 -
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 ---
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 -
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 -
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 -
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 -
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 -
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 -
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 -
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 -
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  657 ----
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 --
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 -
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 -
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 -
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 -
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 -
 .../Impl/Cache/CacheEnumerable.cs               |   82 -
 .../Impl/Cache/CacheEnumerator.cs               |  117 -
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 -
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  940 -----
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 -
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 --
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 -
 .../Impl/Cache/CacheProxyImpl.cs                |  499 ---
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 -
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 -
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 -
 .../Impl/Cache/MutableCacheEntry.cs             |  163 -
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 --
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 -
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 -
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 --
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 -
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 -
 .../Impl/Cache/Query/QueryCursor.cs             |   50 -
 .../Impl/Cache/Store/CacheStore.cs              |  263 --
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 -
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 -
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 ---
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 --
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 --
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 -
 .../Impl/Collections/CollectionExtensions.cs    |   45 -
 .../Impl/Collections/MultiValueDictionary.cs    |  143 -
 .../Impl/Collections/ReadOnlyCollection.cs      |  102 -
 .../Impl/Collections/ReadOnlyDictionary.cs      |  149 -
 .../Impl/Common/AsyncResult.cs                  |   71 -
 .../Impl/Common/CompletedAsyncResult.cs         |   70 -
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   70 -
 .../Impl/Common/DelegateConverter.cs            |  253 --
 .../Impl/Common/DelegateTypeDescriptor.cs       |  314 --
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  286 --
 .../Impl/Common/FutureConverter.cs              |   62 -
 .../Impl/Common/FutureType.cs                   |   52 -
 .../Impl/Common/IFutureConverter.cs             |   34 -
 .../Impl/Common/IFutureInternal.cs              |   46 -
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 -
 .../Impl/Common/LoadedAssembliesResolver.cs     |   96 -
 .../Impl/Common/PortableResultWrapper.cs        |   68 -
 .../Impl/Common/TypeCaster.cs                   |   72 -
 .../Closure/ComputeAbstractClosureTask.cs       |  101 -
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 -
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 -
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 -
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 -
 .../Closure/ComputeReducingClosureTask.cs       |   61 -
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 -
 .../Compute/Closure/IComputeResourceInjector.cs |   31 -
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 -
 .../Impl/Compute/ComputeAsync.cs                |  261 --
 .../Impl/Compute/ComputeFunc.cs                 |  119 -
 .../Impl/Compute/ComputeImpl.cs                 |  645 ---
 .../Impl/Compute/ComputeJob.cs                  |  163 -
 .../Impl/Compute/ComputeJobHolder.cs            |  246 --
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 -
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 -
 .../Impl/Compute/ComputeOutFunc.cs              |  123 -
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 ---
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 --
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 -
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 ----
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 -
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 -
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 ---
 .../Impl/Events/EventsAsync.cs                  |  158 -
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 -
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 -
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   69 -
 .../Impl/Handle/HandleRegistry.cs               |  340 --
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |   35 -
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 -
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  549 ---
 .../Impl/IgniteConfigurationEx.cs               |   57 -
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  490 ---
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 --
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 ---
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 -
 .../InteropDotNetPortableConfiguration.cs       |  127 -
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 -
 .../Impl/InteropExceptionHolder.cs              |   85 -
 .../Impl/LifecycleBeanHolder.cs                 |   66 -
 .../Impl/Memory/IPlatformMemory.cs              |   65 -
 .../Impl/Memory/InteropExternalMemory.cs        |   46 -
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 -
 .../Memory/PlatformBigEndianMemoryStream.cs     |  483 ---
 .../Impl/Memory/PlatformMemory.cs               |   78 -
 .../Impl/Memory/PlatformMemoryManager.cs        |  107 -
 .../Impl/Memory/PlatformMemoryPool.cs           |  106 -
 .../Impl/Memory/PlatformMemoryStream.cs         |  677 ----
 .../Impl/Memory/PlatformMemoryUtils.cs          |  463 ---
 .../Impl/Memory/PlatformPooledMemory.cs         |   70 -
 .../Impl/Memory/PlatformRawMemory.cs            |   89 -
 .../Impl/Memory/PlatformUnpooledMemory.cs       |   52 -
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 -
 .../Impl/Messaging/Messaging.cs                 |  262 --
 .../Impl/Messaging/MessagingAsync.cs            |   68 -
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 -
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ----
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 -
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 -
 .../Impl/Portable/IPortableWriteAware.cs        |   34 -
 .../Impl/Portable/Io/IPortableStream.cs         |  320 --
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1298 -------
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 ---
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 -
 .../Metadata/IPortableMetadataHandler.cs        |   41 -
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 -
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 -
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 -
 .../Impl/Portable/PortableBuilderField.cs       |   73 -
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 -----
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 --
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 -
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 -
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 -
 .../Impl/Portable/PortableMarshaller.cs         |  603 ---
 .../Impl/Portable/PortableMode.cs               |   40 -
 .../Impl/Portable/PortableObjectHandle.cs       |   59 -
 .../PortableOrSerializableObjectHolder.cs       |   66 -
 .../Portable/PortableReaderHandleDictionary.cs  |   42 -
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 -----
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 ---
 .../Portable/PortableReflectiveSerializer.cs    |  218 --
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 -
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 -------
 .../Portable/PortableSystemTypeSerializer.cs    |   62 -
 .../Impl/Portable/PortableUserObject.cs         |  385 --
 .../Impl/Portable/PortableUtils.cs              | 2039 ----------
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 -------
 .../Impl/Portable/PortablesImpl.cs              |  205 -
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 -
 .../Impl/Portable/TypeResolver.cs               |  227 --
 .../Impl/Resource/IResourceInjector.cs          |   27 -
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 -
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 -
 .../Impl/Resource/ResourceProcessor.cs          |  105 -
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 -
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 --
 .../Impl/Services/ServiceContext.cs             |   60 -
 .../Impl/Services/ServiceDescriptor.cs          |  106 -
 .../Impl/Services/ServiceProxy.cs               |   71 -
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 -
 .../Impl/Services/ServiceProxySerializer.cs     |  140 -
 .../Impl/Services/Services.cs                   |  316 --
 .../Impl/Services/ServicesAsync.cs              |   89 -
 .../Impl/Transactions/AsyncTransaction.cs       |   78 -
 .../Impl/Transactions/Transaction.cs            |  155 -
 .../Impl/Transactions/TransactionImpl.cs        |  489 ---
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 -
 .../Impl/Transactions/TransactionsImpl.cs       |  201 -
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 -
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 -
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1154 ------
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 -
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 -
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 -
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1263 ------
 .../Lifecycle/ILifecycleBean.cs                 |   64 -
 .../Lifecycle/LifecycleEventType.cs             |   49 -
 .../Messaging/IMessageFilter.cs                 |   35 -
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 -
 .../Portable/IPortableBuilder.cs                |   78 -
 .../Portable/IPortableIdMapper.cs               |   40 -
 .../Portable/IPortableMarshalAware.cs           |   39 -
 .../Portable/IPortableMetadata.cs               |   61 -
 .../Portable/IPortableNameMapper.cs             |   39 -
 .../Portable/IPortableObject.cs                 |   44 -
 .../Portable/IPortableRawReader.cs              |  264 --
 .../Portable/IPortableRawWriter.cs              |  221 --
 .../Portable/IPortableReader.cs                 |  340 --
 .../Portable/IPortableSerializer.cs             |   39 -
 .../Portable/IPortableWriter.cs                 |  259 --
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 -
 .../Portable/PortableConfiguration.cs           |  122 -
 .../Portable/PortableException.cs               |   64 -
 .../Portable/PortableTypeConfiguration.cs       |  162 -
 .../Portable/PortableTypeNames.cs               |  115 -
 .../Properties/AssemblyInfo.cs                  |   46 -
 .../Resource/InstanceResourceAttribute.cs       |   35 -
 .../Resource/StoreSessionResourceAttribute.cs   |   32 -
 .../Apache.Ignite.Core/Services/IService.cs     |   51 -
 .../Services/IServiceContext.cs                 |   69 -
 .../Services/IServiceDescriptor.cs              |   96 -
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 -
 .../Services/ServiceConfiguration.cs            |   62 -
 .../Services/ServiceInvocationException.cs      |  101 -
 .../Transactions/ITransaction.cs                |  230 --
 .../Transactions/ITransactionMetrics.cs         |   47 -
 .../Transactions/ITransactions.cs               |   73 -
 .../Transactions/TransactionConcurrency.cs      |   36 -
 .../TransactionHeuristicException.cs            |   72 -
 .../Transactions/TransactionIsolation.cs        |   41 -
 .../TransactionOptimisticException.cs           |   69 -
 .../TransactionRollbackException.cs             |   68 -
 .../Transactions/TransactionState.cs            |   70 -
 .../Transactions/TransactionTimeoutException.cs |   69 -
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   68 -
 .../main/dotnet/Apache.Ignite.sln.DotSettings   |    4 -
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   76 -
 .../src/main/dotnet/Apache.Ignite/App.config    |   56 -
 .../Config/AppSettingsConfigurator.cs           |  113 -
 .../Apache.Ignite/Config/ArgsConfigurator.cs    |  164 -
 .../Apache.Ignite/Config/ConfigValueParser.cs   |   42 -
 .../Apache.Ignite/Config/IConfigurator.cs       |   34 -
 .../main/dotnet/Apache.Ignite/IgniteRunner.cs   |  171 -
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   35 -
 .../Apache.Ignite/Service/IgniteService.cs      |  219 --
 .../Apache.Ignite/Service/NativeMethods.cs      |   57 -
 .../Apache.Ignite/Service/ServiceDescription.cs |   32 -
 ...processors.platform.PlatformBootstrapFactory |    2 -
 .../platform/PlatformAbstractBootstrap.java     |   48 -
 .../PlatformAbstractConfigurationClosure.java   |   61 -
 .../platform/PlatformAbstractPredicate.java     |   67 -
 .../platform/PlatformAbstractTarget.java        |  320 --
 .../processors/platform/PlatformBootstrap.java  |   35 -
 .../platform/PlatformBootstrapFactory.java      |   37 -
 .../platform/PlatformConfigurationEx.java       |   48 -
 .../platform/PlatformContextImpl.java           |  621 ---
 .../processors/platform/PlatformIgnition.java   |  189 -
 .../platform/PlatformProcessorImpl.java         |  359 --
 .../platform/cache/PlatformCache.java           | 1090 ------
 .../cache/PlatformCacheEntryFilterImpl.java     |  106 -
 .../cache/PlatformCacheEntryProcessorImpl.java  |  220 --
 .../platform/cache/PlatformCacheIterator.java   |   72 -
 .../PlatformCachePartialUpdateException.java    |   59 -
 .../cache/affinity/PlatformAffinity.java        |  296 --
 .../query/PlatformAbstractQueryCursor.java      |  192 -
 .../query/PlatformContinuousQueryImpl.java      |  235 --
 .../PlatformContinuousQueryRemoteFilter.java    |  188 -
 .../cache/query/PlatformFieldsQueryCursor.java  |   49 -
 .../cache/query/PlatformQueryCursor.java        |   45 -
 .../cache/store/PlatformCacheStoreCallback.java |   61 -
 .../platform/cluster/PlatformClusterGroup.java  |  335 --
 .../cluster/PlatformClusterNodeFilterImpl.java  |   78 -
 .../platform/compute/PlatformAbstractJob.java   |  156 -
 .../platform/compute/PlatformAbstractTask.java  |  206 -
 .../PlatformBalancingMultiClosureTask.java      |   83 -
 ...tformBalancingSingleClosureAffinityTask.java |   88 -
 .../PlatformBalancingSingleClosureTask.java     |   81 -
 .../PlatformBroadcastingMultiClosureTask.java   |   87 -
 .../PlatformBroadcastingSingleClosureTask.java  |   84 -
 .../platform/compute/PlatformClosureJob.java    |  104 -
 .../platform/compute/PlatformCompute.java       |  332 --
 .../platform/compute/PlatformFullJob.java       |  220 --
 .../platform/compute/PlatformFullTask.java      |  192 -
 .../platform/cpp/PlatformCppBootstrap.java      |   31 -
 .../cpp/PlatformCppBootstrapFactory.java        |   39 -
 .../cpp/PlatformCppConfigurationClosure.java    |   99 -
 .../cpp/PlatformCppConfigurationEx.java         |   82 -
 .../datastreamer/PlatformDataStreamer.java      |  226 --
 .../PlatformStreamReceiverImpl.java             |  119 -
 .../dotnet/PlatformDotNetBootstrap.java         |   31 -
 .../dotnet/PlatformDotNetBootstrapFactory.java  |   39 -
 .../dotnet/PlatformDotNetCacheStore.java        |  497 ---
 .../PlatformDotNetConfigurationClosure.java     |  254 --
 .../dotnet/PlatformDotNetConfigurationEx.java   |   90 -
 .../platform/dotnet/PlatformDotNetService.java  |   27 -
 .../dotnet/PlatformDotNetServiceImpl.java       |   47 -
 .../events/PlatformEventFilterListenerImpl.java |  163 -
 .../platform/events/PlatformEvents.java         |  396 --
 .../lifecycle/PlatformLifecycleBean.java        |   75 -
 .../platform/memory/PlatformAbstractMemory.java |  121 -
 .../PlatformBigEndianInputStreamImpl.java       |  126 -
 .../PlatformBigEndianOutputStreamImpl.java      |  161 -
 .../platform/memory/PlatformExternalMemory.java |   55 -
 .../memory/PlatformInputStreamImpl.java         |  331 --
 .../memory/PlatformMemoryManagerImpl.java       |   85 -
 .../platform/memory/PlatformMemoryPool.java     |  140 -
 .../platform/memory/PlatformMemoryUtils.java    |  467 ---
 .../memory/PlatformOutputStreamImpl.java        |  267 --
 .../platform/memory/PlatformPooledMemory.java   |   64 -
 .../platform/memory/PlatformUnpooledMemory.java |   51 -
 .../messaging/PlatformMessageFilterImpl.java    |  110 -
 .../messaging/PlatformMessageLocalFilter.java   |  102 -
 .../platform/messaging/PlatformMessaging.java   |  166 -
 .../services/PlatformAbstractService.java       |  230 --
 .../platform/services/PlatformServices.java     |  275 --
 .../transactions/PlatformTransactions.java      |  259 --
 .../platform/utils/PlatformFutureUtils.java     |  397 --
 .../platform/utils/PlatformReaderBiClosure.java |   34 -
 .../platform/utils/PlatformReaderClosure.java   |   34 -
 .../platform/utils/PlatformUtils.java           |  768 ----
 .../platform/utils/PlatformWriterBiClosure.java |   34 -
 .../platform/utils/PlatformWriterClosure.java   |   33 -
 .../platform/cpp/PlatformCppConfiguration.java  |   47 -
 .../ignite/platform/cpp/package-info.java       |   22 -
 .../dotnet/PlatformDotNetCacheStoreFactory.java |  139 -
 .../dotnet/PlatformDotNetLifecycleBean.java     |  109 -
 .../ignite/platform/dotnet/package-info.java    |   22 -
 .../apache/ignite/platform/package-info.java    |   22 -
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 -
 .../Properties/AssemblyInfo.cs                  |   49 -
 .../TestClass.cs                                |   35 -
 .../Apache.Ignite.Core.Tests.csproj             |  225 --
 .../Cache/CacheAbstractTest.cs                  | 3252 ----------------
 .../Cache/CacheAffinityTest.cs                  |  139 -
 .../Cache/CacheDynamicStartTest.cs              |  281 --
 .../Cache/CacheEntryTest.cs                     |   69 -
 .../Cache/CacheForkedTest.cs                    |   81 -
 .../Cache/CacheLocalAtomicTest.cs               |   57 -
 .../Cache/CacheLocalTest.cs                     |   56 -
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 -
 .../Cache/CachePartitionedAtomicTest.cs         |   50 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 -
 .../Cache/CachePartitionedTest.cs               |   50 -
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 -
 .../Cache/CacheReplicatedTest.cs                |   60 -
 .../Cache/CacheTestAsyncWrapper.cs              |  436 ---
 .../Cache/Query/CacheQueriesTest.cs             |  928 -----
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 ------
 .../ContinuousQueryAtomicBackupTest.cs          |   33 -
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 -
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 -
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 -
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 -
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 -
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 --
 .../Cache/Store/CacheStoreTest.cs               |  510 ---
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 -
 .../Cache/Store/CacheTestStore.cs               |  155 -
 .../Compute/AbstractTaskTest.cs                 |  217 --
 .../Compute/ClosureTaskTest.cs                  |  390 --
 .../Compute/ComputeApiTest.cs                   | 1281 ------
 .../Compute/ComputeMultithreadedTest.cs         |  269 --
 .../Compute/FailoverTaskSelfTest.cs             |  246 --
 .../Forked/ForkedPortableClosureTaskTest.cs     |   30 -
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 -
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 -
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   30 -
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ----
 .../Compute/PortableClosureTaskTest.cs          |  217 --
 .../Compute/PortableTaskTest.cs                 |  253 --
 .../Compute/ResourceTaskTest.cs                 |  568 ---
 .../Compute/SerializableClosureTaskTest.cs      |  217 --
 .../Compute/TaskAdapterTest.cs                  |  274 --
 .../Compute/TaskResultTest.cs                   |  437 ---
 .../Config/Apache.Ignite.exe.config.test        |   41 -
 .../Config/Cache/Store/cache-store-session.xml  |   80 -
 .../Config/Compute/compute-grid1.xml            |   90 -
 .../Config/Compute/compute-grid2.xml            |   63 -
 .../Config/Compute/compute-grid3.xml            |   52 -
 .../Config/Compute/compute-standalone.xml       |   87 -
 .../Config/Dynamic/dynamic-client.xml           |   51 -
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 -
 .../Config/Dynamic/dynamic-data.xml             |   65 -
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 -
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 -
 .../Config/cache-portables.xml                  |   78 -
 .../Config/cache-query-continuous.xml           |  171 -
 .../Config/cache-query.xml                      |  100 -
 .../Config/marshaller-default.xml               |   43 -
 .../Config/marshaller-invalid.xml               |   46 -
 .../Config/marshaller-portable.xml              |   43 -
 .../native-client-test-cache-affinity.xml       |   70 -
 .../native-client-test-cache-parallel-store.xml |   69 -
 .../Config/native-client-test-cache-store.xml   |  125 -
 .../Config/native-client-test-cache.xml         |  194 -
 .../Config/portable.xml                         |   56 -
 .../Config/start-test-grid1.xml                 |   54 -
 .../Config/start-test-grid2.xml                 |   45 -
 .../Config/start-test-grid3.xml                 |   43 -
 .../Dataload/DataStreamerTest.cs                |  592 ---
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 -----
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 --
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  443 ---
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 --
 .../IgniteManagerTest.cs                        |   51 -
 .../IgniteStartStopTest.cs                      |  422 --
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 --
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 --
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 -
 .../Memory/InteropMemoryTest.cs                 |  213 -
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ---
 .../Portable/PortableApiSelfTest.cs             | 1787 ---------
 .../Portable/PortableSelfTest.cs                | 2078 ----------
 .../PortableConfigurationTest.cs                |  173 -
 .../Process/IIgniteProcessOutputReader.cs       |   35 -
 .../Process/IgniteProcess.cs                    |  283 --
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 -
 .../Properties/AssemblyInfo.cs                  |   35 -
 .../Query/ImplicitPortablePerson.cs             |   46 -
 .../Query/NoDefPortablePerson.cs                |   35 -
 .../Query/PortablePerson.cs                     |   69 -
 .../SerializationTest.cs                        |  240 --
 .../Services/ServiceProxyTest.cs                |  741 ----
 .../Services/ServicesAsyncWrapper.cs            |  174 -
 .../Services/ServicesTest.cs                    |  823 ----
 .../Services/ServicesTestAsync.cs               |   33 -
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   71 -
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 --
 .../TypeResolverTest.cs                         |  107 -
 .../platform/PlatformComputeBroadcastTask.java  |   73 -
 .../platform/PlatformComputeDecimalTask.java    |  106 -
 .../platform/PlatformComputeEchoTask.java       |  188 -
 .../ignite/platform/PlatformComputeEnum.java    |   28 -
 .../platform/PlatformComputeJavaPortable.java   |   39 -
 .../platform/PlatformComputePortable.java       |   42 -
 .../PlatformComputePortableArgTask.java         |  121 -
 .../platform/PlatformEventsWriteEventTask.java  |  146 -
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 -
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 -
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 -
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 -
 modules/platforms/cpp/README.txt                |  101 +
 modules/platforms/cpp/common/Makefile.am        |   45 +
 modules/platforms/cpp/common/configure.ac       |   62 +
 .../platforms/cpp/common/ignite-common.pc.in    |    9 +
 .../platforms/cpp/common/include/Makefile.am    |   26 +
 .../common/include/ignite/common/concurrent.h   |  237 ++
 .../cpp/common/include/ignite/common/exports.h  |  158 +
 .../cpp/common/include/ignite/common/java.h     |  679 ++++
 .../cpp/common/include/ignite/common/utils.h    |   81 +
 .../cpp/common/os/linux/include/Makefile.am     |   24 +
 .../os/linux/include/ignite/common/common.h     |   81 +
 .../linux/include/ignite/common/concurrent_os.h |  394 ++
 .../cpp/common/os/linux/src/common.cpp          |   59 +
 .../cpp/common/os/linux/src/concurrent_os.cpp   |  175 +
 .../os/win/include/ignite/common/common.h       |   56 +
 .../win/include/ignite/common/concurrent_os.h   |  406 ++
 .../platforms/cpp/common/os/win/src/common.cpp  |   65 +
 .../cpp/common/os/win/src/concurrent_os.cpp     |  151 +
 modules/platforms/cpp/common/project/README.TXT |    1 +
 .../platforms/cpp/common/project/vs/README.TXT  |    1 +
 .../cpp/common/project/vs/common.vcxproj        |  203 +
 .../common/project/vs/common.vcxproj.filters    |   57 +
 .../platforms/cpp/common/project/vs/module.def  |  111 +
 .../platforms/cpp/common/project/vs/targetver.h |   25 +
 modules/platforms/cpp/common/src/concurrent.cpp |   94 +
 modules/platforms/cpp/common/src/exports.cpp    |  461 +++
 modules/platforms/cpp/common/src/java.cpp       | 2416 ++++++++++++
 modules/platforms/cpp/core-test/Makefile.am     |   49 +
 .../cpp/core-test/config/cache-query.xml        |   91 +
 .../cpp/core-test/config/cache-test.xml         |  129 +
 modules/platforms/cpp/core-test/configure.ac    |   62 +
 .../platforms/cpp/core-test/include/Makefile.am |   22 +
 .../core-test/include/ignite/binary_test_defs.h |  320 ++
 .../include/ignite/binary_test_utils.h          |  516 +++
 .../cpp/core-test/include/teamcity_messages.h   |   55 +
 .../platforms/cpp/core-test/project/README.TXT  |    1 +
 .../cpp/core-test/project/vs/README.TXT         |    1 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  174 +
 .../project/vs/core-test.vcxproj.filters        |   68 +
 .../src/binary_reader_writer_raw_test.cpp       | 1593 ++++++++
 .../core-test/src/binary_reader_writer_test.cpp | 2373 +++++++++++
 .../cpp/core-test/src/binary_session_test.cpp   |  257 ++
 .../cpp/core-test/src/binary_test_defs.cpp      |   65 +
 .../cpp/core-test/src/cache_query_test.cpp      |  841 ++++
 .../platforms/cpp/core-test/src/cache_test.cpp  |  479 +++
 .../cpp/core-test/src/concurrent_test.cpp       |  186 +
 .../cpp/core-test/src/handle_registry_test.cpp  |  176 +
 .../cpp/core-test/src/ignition_test.cpp         |   95 +
 .../cpp/core-test/src/teamcity_boost.cpp        |  159 +
 .../cpp/core-test/src/teamcity_messages.cpp     |  150 +
 modules/platforms/cpp/core/Makefile.am          |   67 +
 modules/platforms/cpp/core/configure.ac         |   62 +
 modules/platforms/cpp/core/ignite.pc.in         |    9 +
 modules/platforms/cpp/core/include/Makefile.am  |   69 +
 .../cpp/core/include/ignite/binary/binary.h     |   29 +
 .../core/include/ignite/binary/binary_consts.h  |  106 +
 .../include/ignite/binary/binary_containers.h   |  525 +++
 .../include/ignite/binary/binary_raw_reader.h   |  350 ++
 .../include/ignite/binary/binary_raw_writer.h   |  326 ++
 .../core/include/ignite/binary/binary_reader.h  |  384 ++
 .../core/include/ignite/binary/binary_type.h    |  293 ++
 .../core/include/ignite/binary/binary_writer.h  |  362 ++
 .../cpp/core/include/ignite/cache/cache.h       | 1196 ++++++
 .../cpp/core/include/ignite/cache/cache_entry.h |  118 +
 .../core/include/ignite/cache/cache_peek_mode.h |   71 +
 .../cpp/core/include/ignite/cache/query/query.h |   28 +
 .../include/ignite/cache/query/query_argument.h |  125 +
 .../include/ignite/cache/query/query_cursor.h   |  200 +
 .../ignite/cache/query/query_fields_cursor.h    |  153 +
 .../ignite/cache/query/query_fields_row.h       |  154 +
 .../include/ignite/cache/query/query_scan.h     |  151 +
 .../core/include/ignite/cache/query/query_sql.h |  243 ++
 .../ignite/cache/query/query_sql_fields.h       |  210 +
 .../include/ignite/cache/query/query_text.h     |  160 +
 .../platforms/cpp/core/include/ignite/guid.h    |  112 +
 .../platforms/cpp/core/include/ignite/ignite.h  |  166 +
 .../core/include/ignite/ignite_configuration.h  |   66 +
 .../cpp/core/include/ignite/ignite_error.h      |  260 ++
 .../cpp/core/include/ignite/ignition.h          |  195 +
 .../include/ignite/impl/binary/binary_common.h  |  188 +
 .../ignite/impl/binary/binary_id_resolver.h     |  106 +
 .../ignite/impl/binary/binary_reader_impl.h     | 1309 +++++++
 .../include/ignite/impl/binary/binary_schema.h  |  136 +
 .../ignite/impl/binary/binary_type_handler.h    |  102 +
 .../ignite/impl/binary/binary_type_manager.h    |  120 +
 .../ignite/impl/binary/binary_type_snapshot.h   |  122 +
 .../ignite/impl/binary/binary_type_updater.h    |   53 +
 .../impl/binary/binary_type_updater_impl.h      |   65 +
 .../include/ignite/impl/binary/binary_utils.h   |  344 ++
 .../ignite/impl/binary/binary_writer_impl.h     |  913 +++++
 .../core/include/ignite/impl/cache/cache_impl.h |  428 ++
 .../impl/cache/query/query_fields_row_impl.h    |  174 +
 .../ignite/impl/cache/query/query_impl.h        |  125 +
 .../core/include/ignite/impl/handle_registry.h  |  202 +
 .../include/ignite/impl/ignite_environment.h    |  130 +
 .../cpp/core/include/ignite/impl/ignite_impl.h  |  165 +
 .../core/include/ignite/impl/interop/interop.h  |   25 +
 .../ignite/impl/interop/interop_input_stream.h  |  250 ++
 .../ignite/impl/interop/interop_memory.h        |  280 ++
 .../ignite/impl/interop/interop_output_stream.h |  250 ++
 .../interop/interop_stream_position_guard.h     |   79 +
 .../cpp/core/include/ignite/impl/operations.h   |  452 +++
 .../cpp/core/os/linux/include/Makefile.am       |   23 +
 .../core/os/linux/include/ignite/impl/utils.h   |  155 +
 .../cpp/core/os/linux/src/impl/utils.cpp        |  439 +++
 .../cpp/core/os/win/include/ignite/impl/utils.h |  155 +
 .../cpp/core/os/win/src/impl/utils.cpp          |  453 +++
 modules/platforms/cpp/core/project/README.TXT   |    1 +
 .../platforms/cpp/core/project/vs/README.TXT    |    1 +
 .../platforms/cpp/core/project/vs/core.vcxproj  |  279 ++
 .../cpp/core/project/vs/core.vcxproj.filters    |  267 ++
 .../cpp/core/src/binary/binary_containers.cpp   |   76 +
 .../cpp/core/src/binary/binary_raw_reader.cpp   |  145 +
 .../cpp/core/src/binary/binary_raw_writer.cpp   |  147 +
 .../cpp/core/src/binary/binary_reader.cpp       |  152 +
 .../cpp/core/src/binary/binary_type.cpp         |   51 +
 .../cpp/core/src/binary/binary_writer.cpp       |  154 +
 modules/platforms/cpp/core/src/guid.cpp         |   65 +
 modules/platforms/cpp/core/src/ignite.cpp       |   43 +
 modules/platforms/cpp/core/src/ignite_error.cpp |  222 ++
 modules/platforms/cpp/core/src/ignition.cpp     |  470 +++
 .../core/src/impl/binary/binary_reader_impl.cpp |  760 ++++
 .../cpp/core/src/impl/binary/binary_schema.cpp  |  135 +
 .../src/impl/binary/binary_type_handler.cpp     |   78 +
 .../src/impl/binary/binary_type_manager.cpp     |  201 +
 .../src/impl/binary/binary_type_snapshot.cpp    |   70 +
 .../src/impl/binary/binary_type_updater.cpp     |   32 +
 .../impl/binary/binary_type_updater_impl.cpp    |   94 +
 .../cpp/core/src/impl/binary/binary_utils.cpp   |  211 +
 .../core/src/impl/binary/binary_writer_impl.cpp |  623 +++
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  393 ++
 .../core/src/impl/cache/query/query_impl.cpp    |  228 ++
 .../cpp/core/src/impl/handle_registry.cpp       |  234 ++
 .../cpp/core/src/impl/ignite_environment.cpp    |  167 +
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   47 +
 .../src/impl/interop/interop_input_stream.cpp   |  235 ++
 .../core/src/impl/interop/interop_memory.cpp    |  182 +
 .../src/impl/interop/interop_output_stream.cpp  |  233 ++
 modules/platforms/cpp/examples/Makefile.am      |   39 +
 modules/platforms/cpp/examples/README.txt       |   42 +
 .../cpp/examples/config/example-cache.xml       |   77 +
 modules/platforms/cpp/examples/configure.ac     |   38 +
 .../platforms/cpp/examples/include/Makefile.am  |   21 +
 .../examples/include/ignite/examples/address.h  |  109 +
 .../include/ignite/examples/organization.h      |  111 +
 .../cpp/examples/project/vs/ignite-examples.sln |   19 +
 .../examples/project/vs/ignite-examples.vcxproj |  107 +
 .../project/vs/ignite-examples.vcxproj.filters  |   30 +
 .../cpp/examples/src/putgetexample.cpp          |  126 +
 modules/platforms/cpp/ignite/Makefile.am        |   39 +
 modules/platforms/cpp/ignite/configure.ac       |   62 +
 modules/platforms/cpp/ignite/project/README.TXT |    1 +
 .../platforms/cpp/ignite/project/vs/README.TXT  |    1 +
 .../cpp/ignite/project/vs/ignite.vcxproj        |  167 +
 .../ignite/project/vs/ignite.vcxproj.filters    |   25 +
 modules/platforms/cpp/ignite/src/ignite.cpp     |  225 ++
 modules/platforms/cpp/project/vs/ignite.sln     |   58 +
 modules/platforms/cpp/project/vs/ignite.slnrel  |   33 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |   33 +
 .../Apache.Ignite.Benchmarks.csproj             |   92 +
 .../Apache.Ignite.Benchmarks.snk                |  Bin 0 -> 596 bytes
 .../dotnet/Apache.Ignite.Benchmarks/App.config  |   24 +
 .../Apache.Ignite.Benchmarks/BenchmarkBase.cs   |  931 +++++
 .../BenchmarkOperationDescriptor.cs             |   68 +
 .../Apache.Ignite.Benchmarks/BenchmarkRunner.cs |   94 +
 .../Apache.Ignite.Benchmarks/BenchmarkState.cs  |  106 +
 .../Apache.Ignite.Benchmarks/BenchmarkUtils.cs  |  236 ++
 .../Binary/BinarizableReadBenchmark.cs          |  125 +
 .../Binary/BinarizableWriteBenchmark.cs         |  135 +
 .../Config/benchmark.xml                        |   57 +
 .../Interop/ClosureBenchmark.cs                 |   66 +
 .../Interop/GetAsyncBenchmark.cs                |   62 +
 .../Interop/GetBenchmark.cs                     |   62 +
 .../Interop/PlatformBenchmarkBase.cs            |  121 +
 .../Interop/PutAsyncBenchmark.cs                |   58 +
 .../Interop/PutBenchmark.cs                     |   58 +
 .../Interop/TaskBenchmark.cs                    |  100 +
 .../Interop/TxBenchmark.cs                      |   65 +
 .../Apache.Ignite.Benchmarks/Model/Address.cs   |   80 +
 .../Apache.Ignite.Benchmarks/Model/Company.cs   |   89 +
 .../Model/Department.cs                         |   40 +
 .../Apache.Ignite.Benchmarks/Model/Employee.cs  |  136 +
 .../Apache.Ignite.Benchmarks/Model/Sex.cs       |   31 +
 .../Apache.Ignite.Benchmarks/Model/TestModel.cs |  111 +
 .../Properties/AssemblyInfo.cs                  |   35 +
 .../Result/BenchmarkConsoleResultWriter.cs      |   68 +
 .../Result/BenchmarkFileResultWriter.cs         |  323 ++
 .../Result/IBenchmarkResultWriter.cs            |   55 +
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   61 +
 .../Apache.Ignite.Core.Tests.TestDll.snk        |  Bin 0 -> 596 bytes
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  259 ++
 .../Apache.Ignite.Core.Tests.nunit              |    7 +
 .../Apache.Ignite.Core.Tests.snk                |  Bin 0 -> 596 bytes
 .../Binary/BinaryBuilderSelfTest.cs             | 1721 ++++++++
 .../Binary/BinarySelfTest.cs                    | 2157 ++++++++++
 .../Binary/BinaryStructureTest.cs               |  250 ++
 .../BinaryConfigurationTest.cs                  |  173 +
 .../Cache/CacheAbstractTest.cs                  | 3265 ++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  282 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   81 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  575 +++
 .../Cache/Query/CacheQueriesTest.cs             |  935 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1238 ++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  536 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/BinarizableClosureTaskTest.cs       |  185 +
 .../Compute/BinarizableTaskTest.cs              |  269 ++
 .../Compute/ClosureTaskTest.cs                  |  390 ++
 .../Compute/ComputeApiTest.cs                   | 1303 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedBinarizableClosureTaskTest.cs  |   30 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   30 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   30 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   30 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  754 ++++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Apache.Ignite.exe.config.test        |   41 +
 .../Config/Cache/Store/cache-store-session.xml  |   79 +
 .../Config/Compute/compute-grid1.xml            |   95 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Apache.Ignite.Core.Tests/Config/binary.xml  |   56 +
 .../Config/cache-binarizables.xml               |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-explicit.xml              |   53 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   68 +
 .../Config/native-client-test-cache-store.xml   |  121 +
 .../Config/native-client-test-cache.xml         |  143 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../DataStructures/AtomicLongTest.cs            |  138 +
 .../Dataload/DataStreamerTest.cs                |  592 +++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  956 +++++
 .../Examples/Example.cs                         |  126 +
 .../Examples/ExamplesTest.cs                    |  156 +
 .../Examples/PathUtil.cs                        |   50 +
 .../Examples/ProjectFilesTest.cs                |   49 +
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  365 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  443 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  188 +
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  426 ++
 .../Apache.Ignite.Core.Tests/IgniteTestBase.cs  |  200 +
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  289 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Memory/InteropMemoryTest.cs                 |  200 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  644 +++
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  283 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Properties/AssemblyInfo.cs                  |   35 +
 .../Query/BinarizablePerson.cs                  |   69 +
 .../Query/ImplicitBinarizablePerson.cs          |   46 +
 .../Query/NoDefBinarizablePerson.cs             |   35 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  182 +
 .../Services/ServicesTest.cs                    |  823 ++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   71 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  306 ++
 .../TypeResolverTest.cs                         |  107 +
 .../Apache.Ignite.Core.csproj                   |  381 ++
 .../Apache.Ignite.Core/Apache.Ignite.Core.snk   |  Bin 0 -> 596 bytes
 .../Binary/BinaryConfiguration.cs               |   90 +
 .../Binary/BinaryObjectException.cs             |   64 +
 .../Binary/BinaryTypeConfiguration.cs           |  116 +
 .../Binary/BinaryTypeNames.cs                   |  121 +
 .../Apache.Ignite.Core/Binary/IBinarizable.cs   |   39 +
 .../Binary/IBinaryIdMapper.cs                   |   40 +
 .../Binary/IBinaryNameMapper.cs                 |   39 +
 .../Apache.Ignite.Core/Binary/IBinaryObject.cs  |   60 +
 .../Binary/IBinaryObjectBuilder.cs              |  310 ++
 .../Binary/IBinaryRawReader.cs                  |  223 ++
 .../Binary/IBinaryRawWriter.cs                  |  220 ++
 .../Apache.Ignite.Core/Binary/IBinaryReader.cs  |  279 ++
 .../Binary/IBinarySerializer.cs                 |   39 +
 .../Apache.Ignite.Core/Binary/IBinaryType.cs    |   52 +
 .../Apache.Ignite.Core/Binary/IBinaryWriter.cs  |  256 ++
 .../Apache.Ignite.Core/Binary/IIgniteBinary.cs  |  120 +
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Apache.Ignite.Core/Cache/CacheResult.cs     |   98 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   45 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   63 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  844 ++++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  158 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   47 +
 .../Cache/Query/IQueryCursor.cs                 |   44 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   76 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  120 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  207 +
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   79 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  235 ++
 .../Cluster/IClusterMetrics.cs                  |  347 ++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  135 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Common/IgniteException.cs                   |   66 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  132 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   95 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  431 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   59 +
 .../Compute/IComputeJobResult.cs                |   66 +
 .../Compute/IComputeReducer.cs                  |   41 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../DataStructures/IAtomicLong.cs               |   84 +
 .../Datastream/IDataStreamer.cs                 |  206 +
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  178 +
 .../Events/CacheQueryExecutedEvent.cs           |   99 +
 .../Events/CacheQueryReadEvent.cs               |  136 +
 .../Events/CacheRebalancingEvent.cs             |  100 +
 .../Events/CheckpointEvent.cs                   |   51 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   82 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  167 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  595 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   33 +
 .../Apache.Ignite.Core/Events/IEventListener.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  259 ++
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  102 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   51 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   93 +
 .../Apache.Ignite.Core/GlobalSuppressions.cs    |  Bin 0 -> 1908 bytes
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  143 +
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  627 +++
 .../Impl/Binary/BinarizableSerializer.cs        |   45 +
 .../Impl/Binary/BinaryBuilderField.cs           |   89 +
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  210 +
 .../Impl/Binary/BinaryHandleDictionary.cs       |  188 +
 .../Impl/Binary/BinaryMode.cs                   |   42 +
 .../Impl/Binary/BinaryObject.cs                 |  354 ++
 .../Impl/Binary/BinaryObjectBuilder.cs          | 1137 ++++++
 .../Impl/Binary/BinaryObjectHandle.cs           |   59 +
 .../Impl/Binary/BinaryObjectHeader.cs           |  496 +++
 .../Impl/Binary/BinaryObjectSchema.cs           |   98 +
 .../Impl/Binary/BinaryObjectSchemaField.cs      |   48 +
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  107 +
 .../Impl/Binary/BinaryReader.cs                 |  965 +++++
 .../Impl/Binary/BinaryReaderExtensions.cs       |   52 +
 .../Impl/Binary/BinaryReaderHandleDictionary.cs |   42 +
 .../Impl/Binary/BinaryReflectiveActions.cs      |  440 +++
 .../Impl/Binary/BinaryReflectiveSerializer.cs   |  218 ++
 .../Binary/BinarySurrogateTypeDescriptor.cs     |  162 +
 .../Impl/Binary/BinarySystemHandlers.cs         |  832 ++++
 .../Impl/Binary/BinarySystemTypeSerializer.cs   |   62 +
 .../Impl/Binary/BinaryUtils.cs                  | 1823 +++++++++
 .../Impl/Binary/BinaryWriter.cs                 | 1428 +++++++
 .../Impl/Binary/DateTimeHolder.cs               |   68 +
 .../Impl/Binary/IBinarySystemTypeSerializer.cs  |   34 +
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |  133 +
 .../Impl/Binary/IBinaryWriteAware.cs            |   34 +
 .../Impl/Binary/IgniteBinary.cs                 |  192 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |  452 +++
 .../Impl/Binary/Io/BinaryStreamAdapter.cs       |  114 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          | 1253 ++++++
 .../Impl/Binary/Io/IBinaryStream.cs             |  322 ++
 .../Impl/Binary/Marshaller.cs                   |  537 +++
 .../Impl/Binary/Metadata/BinaryType.cs          |  200 +
 .../Binary/Metadata/BinaryTypeHashsetHandler.cs |   69 +
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |  147 +
 .../Impl/Binary/Metadata/IBinaryTypeHandler.cs  |   41 +
 .../Impl/Binary/SerializableObjectHolder.cs     |   73 +
 .../Impl/Binary/Structure/BinaryStructure.cs    |  332 ++
 .../Binary/Structure/BinaryStructureEntry.cs    |  128 +
 .../Structure/BinaryStructureJumpTable.cs       |  118 +
 .../Binary/Structure/BinaryStructureTracker.cs  |  140 +
 .../Binary/Structure/BinaryStructureUpdate.cs   |   84 +
 .../Impl/Binary/TypeResolver.cs                 |  231 ++
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  127 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  132 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  144 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  128 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  159 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  | 1255 ++++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   64 +
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   80 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   80 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   85 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |   86 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  213 +
 .../Query/Continuous/ContinuousQueryUtils.cs    |   96 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  574 +++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  294 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  222 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Collections/CollectionExtensions.cs    |   45 +
 .../Impl/Collections/MultiValueDictionary.cs    |  145 +
 .../Impl/Collections/ReadOnlyCollection.cs      |  102 +
 .../Impl/Collections/ReadOnlyDictionary.cs      |  149 +
 .../Apache.Ignite.Core/Impl/Common/Classpath.cs |  159 +
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   72 +
 .../Impl/Common/DelegateConverter.cs            |  269 ++
 .../Impl/Common/DelegateTypeDescriptor.cs       |  340 ++
 .../Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs  |   57 +
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  128 +
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/FutureType.cs                   |   55 +
 .../Impl/Common/IFutureConverter.cs             |   35 +
 .../Impl/Common/IFutureInternal.cs              |   46 +
 .../Impl/Common/IgniteArgumentCheck.cs          |   77 +
 .../Impl/Common/IgniteHome.cs                   |   97 +
 .../Impl/Common/LoadedAssembliesResolver.cs     |   96 +
 .../Impl/Common/ResizeableArray.cs              |   64 +
 .../Impl/Common/TypeCaster.cs                   |   79 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   82 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   86 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   75 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  300 ++
 .../Impl/Compute/ComputeFunc.cs                 |  118 +
 .../Impl/Compute/ComputeImpl.cs                 |  660 ++++
 .../Impl/Compute/ComputeJob.cs                  |  162 +
 .../Impl/Compute/ComputeJobHolder.cs            |  245 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   73 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   99 +
 .../Impl/Compute/ComputeOutFunc.cs              |  122 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  505 +++
 .../Impl/DataStructures/AtomicLong.cs           |  102 +
 .../Impl/Datastream/DataStreamerBatch.cs        |  270 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  840 ++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  648 ++++
 .../Impl/Events/RemoteListenEventFilter.cs      |   84 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  206 +
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   69 +
 .../Impl/Handle/HandleRegistry.cs               |  343 ++
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |   35 +
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  529 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  286 ++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  343 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  418 ++
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/IPlatformMemory.cs              |   65 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Memory/PlatformBigEndianMemoryStream.cs     |  483 +++
 .../Impl/Memory/PlatformMemory.cs               |   78 +
 .../Impl/Memory/PlatformMemoryManager.cs        |  107 +
 .../Impl/Memory/PlatformMemoryPool.cs           |  106 +
 .../Impl/Memory/PlatformMemoryStream.cs         |  727 ++++
 .../Impl/Memory/PlatformMemoryUtils.cs          |  465 +++
 .../Impl/Memory/PlatformPooledMemory.cs         |   66 +
 .../Impl/Memory/PlatformRawMemory.cs            |   89 +
 .../Impl/Memory/PlatformUnpooledMemory.cs       |   52 +
 .../Impl/Messaging/MessageListenerHolder.cs     |  177 +
 .../Impl/Messaging/Messaging.cs                 |  289 ++
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  737 ++++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  141 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  372 ++
 .../Impl/Transactions/Transaction.cs            |  146 +
 .../Impl/Transactions/TransactionImpl.cs        |  482 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   68 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 +
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1164 ++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   70 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1371 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageListener.cs               |   38 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  128 +
 .../Properties/AssemblyInfo.cs                  |   46 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  256 ++
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  238 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   76 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 modules/platforms/dotnet/Apache.Ignite.FxCop    |  117 +
 modules/platforms/dotnet/Apache.Ignite.sln      |  111 +
 .../dotnet/Apache.Ignite.sln.DotSettings        |    4 +
 modules/platforms/dotnet/Apache.Ignite.slnrel   |   43 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   83 +
 .../dotnet/Apache.Ignite/Apache.Ignite.snk      |  Bin 0 -> 596 bytes
 .../platforms/dotnet/Apache.Ignite/App.config   |   56 +
 .../Config/AppSettingsConfigurator.cs           |  113 +
 .../Apache.Ignite/Config/ArgsConfigurator.cs    |  164 +
 .../Apache.Ignite/Config/ConfigValueParser.cs   |   42 +
 .../Apache.Ignite/Config/IConfigurator.cs       |   34 +
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |  171 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   35 +
 .../Apache.Ignite/Service/IgniteService.cs      |  219 ++
 .../Apache.Ignite/Service/NativeMethods.cs      |   57 +
 .../Apache.Ignite/Service/ServiceDescription.cs |   32 +
 .../platforms/dotnet/Apache.Ignite_x86.slnrel   |   43 +
 modules/platforms/dotnet/README.txt             |   24 +
 .../dotnet/examples/Apache.Ignite.Examples.sln  |   38 +
 .../Apache.Ignite.Examples.csproj               |   88 +
 .../Apache.Ignite.Examples.snk                  |  Bin 0 -> 596 bytes
 .../examples/Apache.Ignite.Examples/App.config  |   24 +
 .../Compute/ClosureExample.cs                   |   84 +
 .../Compute/TaskExample.cs                      |  141 +
 .../Datagrid/ContinuousQueryExample.cs          |  103 +
 .../Datagrid/CrossPlatformExample.cs            |  205 +
 .../Datagrid/DataStreamerExample.cs             |  102 +
 .../Datagrid/PutGetExample.cs                   |  219 ++
 .../Datagrid/QueryExample.cs                    |  227 ++
 .../Datagrid/StoreExample.cs                    |  115 +
 .../Datagrid/TransactionExample.cs              |  105 +
 .../Events/EventsExample.cs                     |  104 +
 .../Messaging/MessagingExample.cs               |  112 +
 .../Misc/LifecycleExample.cs                    |  109 +
 .../Properties/AssemblyInfo.cs                  |   35 +
 .../Services/IMapService.cs                     |   56 +
 .../Services/ServicesExample.cs                 |   77 +
 .../Apache.Ignite.ExamplesDll.csproj            |   82 +
 .../Apache.Ignite.ExamplesDll.snk               |  Bin 0 -> 596 bytes
 .../Apache.Ignite.ExamplesDll/Binary/Account.cs |   60 +
 .../Apache.Ignite.ExamplesDll/Binary/Address.cs |   81 +
 .../Binary/Employee.cs                          |   93 +
 .../Binary/EmployeeKey.cs                       |   86 +
 .../Binary/Organization.cs                      |   84 +
 .../Binary/OrganizationType.cs                  |   43 +
 .../Compute/AverageSalaryJob.cs                 |   66 +
 .../Compute/AverageSalaryTask.cs                |   85 +
 .../Compute/CharacterCountClosure.cs            |   43 +
 .../Compute/CharacterCountReducer.cs            |   51 +
 .../Datagrid/ContinuousQueryFilter.cs           |   50 +
 .../Datagrid/EmployeeStore.cs                   |  122 +
 .../Datagrid/EmployeeStorePredicate.cs          |   41 +
 .../Events/LocalListener.cs                     |   54 +
 .../Messaging/LocalListener.cs                  |   59 +
 .../Messaging/RemoteOrderedListener.cs          |   54 +
 .../Messaging/RemoteUnorderedListener.cs        |   54 +
 .../Messaging/Topic.cs                          |   28 +
 .../Properties/AssemblyInfo.cs                  |   35 +
 .../Services/MapService.cs                      |  119 +
 .../examples/Config/example-cache-query.xml     |  111 +
 .../examples/Config/example-cache-store.xml     |   59 +
 .../dotnet/examples/Config/example-cache.xml    |   80 +
 .../dotnet/examples/Config/example-compute.xml  |   70 +
 modules/platforms/dotnet/examples/README.txt    |   14 +
 modules/platforms/licenses/apache-2.0.txt       |  202 +
 .../org/apache/ignite/IgniteSpringBean.java     |    7 +
 modules/yardstick/config/ignite-base-config.xml |   37 +-
 .../cache/IgniteSqlQueryPutBenchmark.java       |    5 +
 .../yardstick/cache/model/Organization.java     |   19 +-
 .../ignite/yardstick/cache/model/Person.java    |   24 +-
 .../ignite/yardstick/cache/model/SampleKey.java |   16 +-
 .../yardstick/cache/model/SampleValue.java      |   16 +-
 parent/pom.xml                                  |   34 +-
 pom.xml                                         |   11 -
 1760 files changed, 173956 insertions(+), 140531 deletions(-)
----------------------------------------------------------------------



[7/8] ignite git commit: Merge branch ignite-1.5 into ignite-1.5-tx-futs-opts

Posted by ag...@apache.org.
Merge branch ignite-1.5 into ignite-1.5-tx-futs-opts


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

Branch: refs/heads/ignite-1.5
Commit: 388c85785271deabc8822e1d8cf1fdaf4254b6e2
Parents: 7cd4a77
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 09:36:32 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 09:36:32 2015 +0300

----------------------------------------------------------------------
 .../colocated/GridDhtColocatedLockFuture.java   | 61 +++++++-------------
 .../distributed/near/GridNearLockFuture.java    | 18 +++---
 2 files changed, 29 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/388c8578/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index bd6c2a7..7e6ce89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -89,26 +89,26 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
 
     /** Cache registry. */
     @GridToStringExclude
-    private GridCacheContext<?, ?> cctx;
+    private final GridCacheContext<?, ?> cctx;
 
     /** Lock owner thread. */
     @GridToStringInclude
-    private long threadId;
+    private final long threadId;
 
     /** Keys to lock. */
     private Collection<KeyCacheObject> keys;
 
     /** Future ID. */
-    private IgniteUuid futId;
+    private final IgniteUuid futId;
 
     /** Lock version. */
-    private GridCacheVersion lockVer;
+    private final GridCacheVersion lockVer;
 
     /** Read flag. */
-    private boolean read;
+    private final boolean read;
 
     /** Flag to return value. */
-    private boolean retval;
+    private final boolean retval;
 
     /** Error. */
     private volatile Throwable err;
@@ -118,26 +118,26 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     private LockTimeoutObject timeoutObj;
 
     /** Lock timeout. */
-    private long timeout;
+    private final long timeout;
 
     /** Filter. */
-    private CacheEntryPredicate[] filter;
+    private final CacheEntryPredicate[] filter;
 
     /** Transaction. */
     @GridToStringExclude
-    private GridNearTxLocal tx;
+    private final GridNearTxLocal tx;
 
     /** Topology snapshot to operate on. */
     private volatile AffinityTopologyVersion topVer;
 
     /** Map of current values. */
-    private Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
+    private final Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
 
     /** Trackable flag (here may be non-volatile). */
     private boolean trackable;
 
     /** TTL for read operation. */
-    private long accessTtl;
+    private final long accessTtl;
 
     /** Skip store flag. */
     private final boolean skipStore;
@@ -457,7 +457,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      * @param miniId Mini ID to find.
      * @return Mini future.
      */
-    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @SuppressWarnings({"ForLoopReplaceableByForEach", "IfMayBeConditional"})
     private MiniFuture miniFuture(IgniteUuid miniId) {
         // We iterate directly over the futs collection here to avoid copy.
         synchronized (futs) {
@@ -742,6 +742,12 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         }
     }
 
+    /**
+     * @param keys Keys to map.
+     * @param remap Remap flag.
+     * @param topLocked Topology locked flag.
+     * @throws IgniteCheckedException If mapping failed.
+     */
     private synchronized void map0(
         Collection<KeyCacheObject> keys,
         boolean remap,
@@ -908,34 +914,6 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                                     mapping.request(req);
                                 }
 
-                                req = new GridNearLockRequest(
-                                    cctx.cacheId(),
-                                    topVer,
-                                    cctx.nodeId(),
-                                    threadId,
-                                    futId,
-                                    lockVer,
-                                    inTx(),
-                                    implicitTx(),
-                                    implicitSingleTx(),
-                                    read,
-                                    retval,
-                                    isolation(),
-                                    isInvalidate(),
-                                    timeout,
-                                    mappedKeys.size(),
-                                    inTx() ? tx.size() : mappedKeys.size(),
-                                    inTx() && tx.syncCommit(),
-                                    inTx() ? tx.subjectId() : null,
-                                    inTx() ? tx.taskNameHash() : 0,
-                                    read ? accessTtl : -1L,
-                                    skipStore,
-                                    clientFirst,
-                                    cctx.deploymentEnabled());
-
-                                mapping.request(req);
-                            }
-
                             distributedKeys.add(key);
 
                             if (tx != null)
@@ -1502,7 +1480,8 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                             false,
                             CU.subjectId(tx, cctx.shared()),
                             null,
-                            tx == null ? null : tx.resolveTaskName());
+                            tx == null ? null : tx.resolveTaskName(),
+                            keepBinary);
                     }
 
                     i++;

http://git-wip-us.apache.org/repos/asf/ignite/blob/388c8578/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 832cc3d..f1f9990 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -85,26 +85,26 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
 
     /** Cache registry. */
     @GridToStringExclude
-    private GridCacheContext<?, ?> cctx;
+    private final GridCacheContext<?, ?> cctx;
 
     /** Lock owner thread. */
     @GridToStringInclude
     private long threadId;
 
     /** Keys to lock. */
-    private Collection<KeyCacheObject> keys;
+    private final Collection<KeyCacheObject> keys;
 
     /** Future ID. */
-    private IgniteUuid futId;
+    private final IgniteUuid futId;
 
     /** Lock version. */
-    private GridCacheVersion lockVer;
+    private final GridCacheVersion lockVer;
 
     /** Read flag. */
     private boolean read;
 
     /** Flag to return value. */
-    private boolean retval;
+    private final boolean retval;
 
     /** Error. */
     private volatile Throwable err;
@@ -120,17 +120,17 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     private long timeout;
 
     /** Filter. */
-    private CacheEntryPredicate[] filter;
+    private final CacheEntryPredicate[] filter;
 
     /** Transaction. */
     @GridToStringExclude
-    private GridNearTxLocal tx;
+    private final GridNearTxLocal tx;
 
     /** Topology snapshot to operate on. */
     private volatile AffinityTopologyVersion topVer;
 
     /** Map of current values. */
-    private Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
+    private final Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
 
     /** Trackable flag. */
     private boolean trackable = true;
@@ -496,7 +496,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
      * @param miniId Mini ID to find.
      * @return Mini future.
      */
-    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @SuppressWarnings({"ForLoopReplaceableByForEach", "IfMayBeConditional"})
     private MiniFuture miniFuture(IgniteUuid miniId) {
         // We iterate directly over the futs collection here to avoid copy.
         synchronized (futs) {


[3/8] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5-tx-futs-opts

Posted by ag...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5-tx-futs-opts


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

Branch: refs/heads/ignite-1.5
Commit: d92de42de528893920f51711acaa66b6d882549f
Parents: 5e6d0ff ab8298a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Nov 20 19:18:01 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Nov 20 19:18:01 2015 +0300

----------------------------------------------------------------------
 .../stream/camel/IgniteCamelStreamerTest.java   |  16 +-
 .../camel/IgniteCamelStreamerTestSuite.java     |   1 -
 .../internal/MarshallerContextAdapter.java      |   4 +-
 .../GridCachePartitionExchangeManager.java      |  34 ++-
 .../dht/GridClientPartitionTopology.java        |  38 +--
 .../dht/GridDhtPartitionTopology.java           |  12 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  45 +--
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 -
 .../dht/GridPartitionedGetFuture.java           |   2 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  41 ++-
 .../dht/preloader/GridDhtPartitionMap.java      | 148 +--------
 .../dht/preloader/GridDhtPartitionMap2.java     | 306 +++++++++++++++++++
 .../GridDhtPartitionsExchangeFuture.java        |  24 +-
 .../GridDhtPartitionsSingleMessage.java         |   6 +-
 .../distributed/near/GridNearCacheAdapter.java  |   1 -
 ...arOptimisticSerializableTxPrepareFuture.java |   3 -
 .../near/GridNearOptimisticTxPrepareFuture.java |   1 -
 .../near/GridNearTxFinishFuture.java            |   4 -
 .../distributed/near/GridNearTxRemote.java      |   1 -
 .../processors/cache/local/GridLocalCache.java  |   1 -
 .../cache/local/GridLocalLockFuture.java        |   2 -
 .../cache/transactions/IgniteInternalTx.java    |   1 -
 .../cache/transactions/IgniteTxAdapter.java     |   1 -
 .../transactions/IgniteTxLocalAdapter.java      |   1 -
 .../cache/transactions/IgniteTxManager.java     |   1 -
 .../cache/transactions/IgniteTxStateImpl.java   |   1 -
 .../ignite/internal/visor/cache/VisorCache.java |   6 +-
 .../org/apache/ignite/stream/StreamAdapter.java |   1 -
 .../resources/META-INF/classnames.properties    |   1 +
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  12 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   8 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java | 178 +++++++++++
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 .../h2/twostep/GridReduceQueryExecutor.java     |   4 +-
 .../cache/WaitMapExchangeFinishCallable.java    |   4 +-
 .../IgniteFailoverAbstractBenchmark.java        |   4 +-
 37 files changed, 661 insertions(+), 259 deletions(-)
----------------------------------------------------------------------



[5/8] ignite git commit: Merge branch ignite-1.5 into ignite-1.5-tx-futs-opts

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7cd4a77a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7cd4a77a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 7e17efe,b266ad2..bd6c2a7
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@@ -142,9 -143,9 +142,12 @@@ public final class GridDhtColocatedLock
      /** Skip store flag. */
      private final boolean skipStore;
  
 +    /** */
 +    private Deque<GridNearLockMapping> mappings;
 +
+     /** Keep binary. */
+     private final boolean keepBinary;
+ 
      /**
       * @param cctx Registry.
       * @param keys Keys to lock.
@@@ -727,213 -720,214 +732,242 @@@
       */
      private void map(Collection<KeyCacheObject> keys, boolean remap, boolean topLocked) {
          try {
 -            AffinityTopologyVersion topVer = this.topVer.get();
 +            map0(
 +                keys,
 +                remap,
 +                topLocked);
 +        }
 +        catch (IgniteCheckedException ex) {
 +            onDone(false, ex);
 +        }
 +    }
  
 -            assert topVer != null;
 +    private synchronized void map0(
 +        Collection<KeyCacheObject> keys,
 +        boolean remap,
 +        boolean topLocked
 +    ) throws IgniteCheckedException {
 +        AffinityTopologyVersion topVer = this.topVer;
  
 -            assert topVer.topologyVersion() > 0;
 +        assert topVer != null;
  
 -            if (CU.affinityNodes(cctx, topVer).isEmpty()) {
 -                onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
 -                    "(all partition nodes left the grid): " + cctx.name()));
 +        assert topVer.topologyVersion() > 0;
  
 -                return;
 -            }
 +        if (CU.affinityNodes(cctx, topVer).isEmpty()) {
 +            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
 +                "(all partition nodes left the grid): " + cctx.name()));
  
 -            boolean clientNode = cctx.kernalContext().clientNode();
 +            return;
 +        }
  
 -            assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
 +        boolean clientNode = cctx.kernalContext().clientNode();
  
 -            // First assume this node is primary for all keys passed in.
 -            if (!clientNode && mapAsPrimary(keys, topVer))
 -                return;
 +        assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
  
 -            Deque<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
 +        // First assume this node is primary for all keys passed in.
 +        if (!clientNode && mapAsPrimary(keys, topVer))
 +            return;
  
 -            // Assign keys to primary nodes.
 -            GridNearLockMapping map = null;
 +        mappings = new ArrayDeque<>();
  
 -            for (KeyCacheObject key : keys) {
 -                GridNearLockMapping updated = map(key, map, topVer);
 +        // Assign keys to primary nodes.
 +        GridNearLockMapping map = null;
  
 -                // If new mapping was created, add to collection.
 -                if (updated != map) {
 -                    mappings.add(updated);
 +        for (KeyCacheObject key : keys) {
 +            GridNearLockMapping updated = map(key, map, topVer);
  
 -                    if (tx != null && updated.node().isLocal())
 -                        tx.colocatedLocallyMapped(true);
 -                }
 +            // If new mapping was created, add to collection.
 +            if (updated != map) {
 +                mappings.add(updated);
  
 -                map = updated;
 +                if (tx != null && updated.node().isLocal())
 +                    tx.colocatedLocallyMapped(true);
              }
  
 -            if (isDone()) {
 -                if (log.isDebugEnabled())
 -                    log.debug("Abandoning (re)map because future is done: " + this);
 -
 -                return;
 -            }
 +            map = updated;
 +        }
  
 +        if (isDone()) {
              if (log.isDebugEnabled())
 -                log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
 +                log.debug("Abandoning (re)map because future is done: " + this);
 +
 +            return;
 +        }
  
 -            boolean hasRmtNodes = false;
 +        if (log.isDebugEnabled())
 +            log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
  
 -            boolean first = true;
 +        boolean hasRmtNodes = false;
  
 -            // Create mini futures.
 -            for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
 -                GridNearLockMapping mapping = iter.next();
 +        boolean first = true;
  
 -                ClusterNode node = mapping.node();
 -                Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
 +        // Create mini futures.
 +        for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
 +            GridNearLockMapping mapping = iter.next();
  
 -                boolean loc = node.equals(cctx.localNode());
 +            ClusterNode node = mapping.node();
 +            Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
  
 -                assert !mappedKeys.isEmpty();
 +            boolean loc = node.equals(cctx.localNode());
  
 -                GridNearLockRequest req = null;
 +            assert !mappedKeys.isEmpty();
  
 -                Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
 +            GridNearLockRequest req = null;
  
 -                for (KeyCacheObject key : mappedKeys) {
 -                    IgniteTxKey txKey = cctx.txKey(key);
 +            Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
  
 -                    GridDistributedCacheEntry entry = null;
 +            for (KeyCacheObject key : mappedKeys) {
 +                IgniteTxKey txKey = cctx.txKey(key);
  
 -                    if (tx != null) {
 -                        IgniteTxEntry txEntry = tx.entry(txKey);
 +                GridDistributedCacheEntry entry = null;
  
 -                        if (txEntry != null) {
 -                            entry = (GridDistributedCacheEntry)txEntry.cached();
 +                if (tx != null) {
 +                    IgniteTxEntry txEntry = tx.entry(txKey);
  
 -                            if (entry != null && !(loc ^ entry.detached())) {
 -                                entry = cctx.colocated().entryExx(key, topVer, true);
 +                    if (txEntry != null) {
 +                        entry = (GridDistributedCacheEntry)txEntry.cached();
  
 -                                txEntry.cached(entry);
 -                            }
 +                        if (entry != null && !(loc ^ entry.detached())) {
 +                            entry = cctx.colocated().entryExx(key, topVer, true);
 +
 +                            txEntry.cached(entry);
                          }
                      }
 +                }
  
 -                    boolean explicit;
 +                boolean explicit;
  
 -                    while (true) {
 -                        try {
 -                            if (entry == null)
 -                                entry = cctx.colocated().entryExx(key, topVer, true);
 +                while (true) {
 +                    try {
 +                        if (entry == null)
 +                            entry = cctx.colocated().entryExx(key, topVer, true);
  
 -                            if (!cctx.isAll(entry, filter)) {
 -                                if (log.isDebugEnabled())
 -                                    log.debug("Entry being locked did not pass filter (will not lock): " + entry);
 +                        if (!cctx.isAll(entry, filter)) {
 +                            if (log.isDebugEnabled())
 +                                log.debug("Entry being locked did not pass filter (will not lock): " + entry);
  
 -                                onComplete(false, false);
 +                            onComplete(false, false);
  
 -                                return;
 -                            }
 +                            return;
 +                        }
  
 -                            assert loc ^ entry.detached() : "Invalid entry [loc=" + loc + ", entry=" + entry + ']';
 +                        assert loc ^ entry.detached() : "Invalid entry [loc=" + loc + ", entry=" + entry + ']';
  
 -                            GridCacheMvccCandidate cand = addEntry(entry);
 +                        GridCacheMvccCandidate cand = addEntry(entry);
  
 -                            // Will either return value from dht cache or null if this is a miss.
 -                            IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.detached() ? null :
 -                                ((GridDhtCacheEntry)entry).versionedValue(topVer);
 +                        // Will either return value from dht cache or null if this is a miss.
 +                        IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.detached() ? null :
 +                            ((GridDhtCacheEntry)entry).versionedValue(topVer);
  
 -                            GridCacheVersion dhtVer = null;
 +                        GridCacheVersion dhtVer = null;
  
 -                            if (val != null) {
 -                                dhtVer = val.get1();
 +                        if (val != null) {
 +                            dhtVer = val.get1();
  
 -                                valMap.put(key, val);
 -                            }
 +                            valMap.put(key, val);
 +                        }
  
 -                            if (cand != null && !cand.reentry()) {
 -                                if (req == null) {
 -                                    boolean clientFirst = false;
 +                        if (cand != null && !cand.reentry()) {
 +                            if (req == null) {
 +                                boolean clientFirst = false;
  
 -                                    if (first) {
 -                                        clientFirst = clientNode &&
 -                                            !topLocked &&
 -                                            (tx == null || !tx.hasRemoteLocks());
 +                                if (first) {
 +                                    clientFirst = clientNode &&
 +                                        !topLocked &&
 +                                        (tx == null || !tx.hasRemoteLocks());
  
-                                     first = false;
+                                         first = false;
+                                     }
+ 
+                                     req = new GridNearLockRequest(
+                                         cctx.cacheId(),
+                                         topVer,
+                                         cctx.nodeId(),
+                                         threadId,
+                                         futId,
+                                         lockVer,
+                                         inTx(),
+                                         implicitTx(),
+                                         implicitSingleTx(),
+                                         read,
+                                         retval,
+                                         isolation(),
+                                         isInvalidate(),
+                                         timeout,
+                                         mappedKeys.size(),
+                                         inTx() ? tx.size() : mappedKeys.size(),
+                                         inTx() && tx.syncCommit(),
+                                         inTx() ? tx.subjectId() : null,
+                                         inTx() ? tx.taskNameHash() : 0,
+                                         read ? accessTtl : -1L,
+                                         skipStore,
+                                         keepBinary,
+                                         clientFirst,
+                                         cctx.deploymentEnabled());
+ 
+                                     mapping.request(req);
                                  }
  
 -                                distributedKeys.add(key);
 -
 -                                if (tx != null)
 -                                    tx.addKeyMapping(txKey, mapping.node());
 -
 -                                req.addKeyBytes(
 -                                    key,
 +                                req = new GridNearLockRequest(
 +                                    cctx.cacheId(),
 +                                    topVer,
 +                                    cctx.nodeId(),
 +                                    threadId,
 +                                    futId,
 +                                    lockVer,
 +                                    inTx(),
 +                                    implicitTx(),
 +                                    implicitSingleTx(),
 +                                    read,
                                      retval,
 -                                    dhtVer, // Include DHT version to match remote DHT entry.
 -                                    cctx);
 +                                    isolation(),
 +                                    isInvalidate(),
 +                                    timeout,
 +                                    mappedKeys.size(),
 +                                    inTx() ? tx.size() : mappedKeys.size(),
 +                                    inTx() && tx.syncCommit(),
 +                                    inTx() ? tx.subjectId() : null,
 +                                    inTx() ? tx.taskNameHash() : 0,
 +                                    read ? accessTtl : -1L,
 +                                    skipStore,
 +                                    clientFirst,
 +                                    cctx.deploymentEnabled());
 +
 +                                mapping.request(req);
                              }
  
 -                            explicit = inTx() && cand == null;
 +                            distributedKeys.add(key);
  
 -                            if (explicit)
 +                            if (tx != null)
                                  tx.addKeyMapping(txKey, mapping.node());
  
 -                            break;
 +                            req.addKeyBytes(
 +                                key,
 +                                retval,
 +                                dhtVer, // Include DHT version to match remote DHT entry.
 +                                cctx);
                          }
 -                        catch (GridCacheEntryRemovedException ignored) {
 -                            if (log.isDebugEnabled())
 -                                log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
  
 -                            entry = null;
 -                        }
 -                    }
 +                        explicit = inTx() && cand == null;
  
 -                    // Mark mapping explicit lock flag.
 -                    if (explicit) {
 -                        boolean marked = tx != null && tx.markExplicit(node.id());
 +                        if (explicit)
 +                            tx.addKeyMapping(txKey, mapping.node());
  
 -                        assert tx == null || marked;
 +                        break;
                      }
 -                }
 -
 -                if (inTx() && req != null)
 -                    req.hasTransforms(tx.hasTransforms());
 +                    catch (GridCacheEntryRemovedException ignored) {
 +                        if (log.isDebugEnabled())
 +                            log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
  
 -                if (!distributedKeys.isEmpty()) {
 -                    mapping.distributedKeys(distributedKeys);
 -
 -                    hasRmtNodes |= !mapping.node().isLocal();
 +                        entry = null;
 +                    }
                  }
 -                else {
 -                    assert mapping.request() == null;
  
 -                    iter.remove();
 +                // Mark mapping explicit lock flag.
 +                if (explicit) {
 +                    boolean marked = tx != null && tx.markExplicit(node.id());
 +
 +                    assert tx == null || marked;
                  }
              }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/7cd4a77a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 6f92204,f16573d..832cc3d
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@@ -146,10 -150,9 +146,13 @@@ public final class GridNearLockFuture e
      /** Skip store flag. */
      private final boolean skipStore;
  
 +    /** Mappings to proceed. */
 +    @GridToStringExclude
 +    private Queue<GridNearLockMapping> mappings;
 +
+     /** Keep binary context flag. */
+     private final boolean keepBinary;
+ 
      /**
       * @param cctx Registry.
       * @param keys Keys to lock.
@@@ -848,221 -845,199 +853,222 @@@
  
              assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
  
 -            ConcurrentLinkedDeque8<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
 +            synchronized (this) {
 +                mappings = new ArrayDeque<>();
  
 -            // Assign keys to primary nodes.
 -            GridNearLockMapping map = null;
 +                // Assign keys to primary nodes.
 +                GridNearLockMapping map = null;
  
 -            for (KeyCacheObject key : keys) {
 -                GridNearLockMapping updated = map(key, map, topVer);
 +                for (KeyCacheObject key : keys) {
 +                    GridNearLockMapping updated = map(
 +                        key,
 +                        map,
 +                        topVer);
  
 -                // If new mapping was created, add to collection.
 -                if (updated != map) {
 -                    mappings.add(updated);
 +                    // If new mapping was created, add to collection.
 +                    if (updated != map) {
 +                        mappings.add(updated);
  
 -                    if (tx != null && updated.node().isLocal())
 -                        tx.nearLocallyMapped(true);
 +                        if (tx != null && updated.node().isLocal())
 +                            tx.nearLocallyMapped(true);
 +                    }
 +
 +                    map = updated;
                  }
  
 -                map = updated;
 -            }
 +                if (isDone()) {
 +                    if (log.isDebugEnabled())
 +                        log.debug("Abandoning (re)map because future is done: " + this);
  
 -            if (isDone()) {
 -                if (log.isDebugEnabled())
 -                    log.debug("Abandoning (re)map because future is done: " + this);
 +                    return;
 +                }
  
 -                return;
 -            }
 +                if (log.isDebugEnabled())
 +                    log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
  
 -            if (log.isDebugEnabled())
 -                log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
 +                boolean first = true;
  
 -            boolean first = true;
 +                // Create mini futures.
 +                for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
 +                    GridNearLockMapping mapping = iter.next();
  
 -            // Create mini futures.
 -            for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
 -                GridNearLockMapping mapping = iter.next();
 +                    ClusterNode node = mapping.node();
 +                    Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
  
 -                ClusterNode node = mapping.node();
 -                Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
 +                    assert !mappedKeys.isEmpty();
  
 -                assert !mappedKeys.isEmpty();
 +                    GridNearLockRequest req = null;
  
 -                GridNearLockRequest req = null;
 +                    Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
  
 -                Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
 +                    boolean explicit = false;
  
 -                boolean explicit = false;
 +                    for (KeyCacheObject key : mappedKeys) {
 +                        IgniteTxKey txKey = cctx.txKey(key);
  
 -                for (KeyCacheObject key : mappedKeys) {
 -                    IgniteTxKey txKey = cctx.txKey(key);
 +                        while (true) {
 +                            GridNearCacheEntry entry = null;
  
 -                    while (true) {
 -                        GridNearCacheEntry entry = null;
 +                            try {
 +                                entry = cctx.near().entryExx(
 +                                    key,
 +                                    topVer);
  
 -                        try {
 -                            entry = cctx.near().entryExx(key, topVer);
 +                                if (!cctx.isAll(
 +                                    entry,
 +                                    filter)) {
 +                                    if (log.isDebugEnabled())
 +                                        log.debug("Entry being locked did not pass filter (will not lock): " + entry);
  
 -                            if (!cctx.isAll(entry, filter)) {
 -                                if (log.isDebugEnabled())
 -                                    log.debug("Entry being locked did not pass filter (will not lock): " + entry);
 +                                    onComplete(
 +                                        false,
 +                                        false);
  
 -                                onComplete(false, false);
 +                                    return;
 +                                }
  
 -                                return;
 -                            }
 +                                // Removed exception may be thrown here.
 +                                GridCacheMvccCandidate cand = addEntry(
 +                                    topVer,
 +                                    entry,
 +                                    node.id());
  
 -                            // Removed exception may be thrown here.
 -                            GridCacheMvccCandidate cand = addEntry(topVer, entry, node.id());
 +                                if (isDone()) {
 +                                    if (log.isDebugEnabled())
 +                                        log.debug("Abandoning (re)map because future is done after addEntry attempt " +
 +                                            "[fut=" + this + ", entry=" + entry + ']');
  
 -                            if (isDone()) {
 -                                if (log.isDebugEnabled())
 -                                    log.debug("Abandoning (re)map because future is done after addEntry attempt " +
 -                                        "[fut=" + this + ", entry=" + entry + ']');
 +                                    return;
 +                                }
  
 -                                return;
 -                            }
 +                                if (cand != null) {
 +                                    if (tx == null && !cand.reentry())
 +                                        cctx.mvcc().addExplicitLock(
 +                                            threadId,
 +                                            cand,
 +                                            topVer);
  
 -                            if (cand != null) {
 -                                if (tx == null && !cand.reentry())
 -                                    cctx.mvcc().addExplicitLock(threadId, cand, topVer);
 +                                    IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.versionedValue();
  
 -                                IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.versionedValue();
 +                                    if (val == null) {
 +                                        GridDhtCacheEntry dhtEntry = dht().peekExx(key);
  
 -                                if (val == null) {
 -                                    GridDhtCacheEntry dhtEntry = dht().peekExx(key);
 +                                        try {
 +                                            if (dhtEntry != null)
 +                                                val = dhtEntry.versionedValue(topVer);
 +                                        }
 +                                        catch (GridCacheEntryRemovedException ignored) {
 +                                            assert dhtEntry.obsolete() : dhtEntry;
  
 -                                    try {
 -                                        if (dhtEntry != null)
 -                                            val = dhtEntry.versionedValue(topVer);
 +                                            if (log.isDebugEnabled())
 +                                                log.debug("Got removed exception for DHT entry in map (will ignore): "
 +                                                    + dhtEntry);
 +                                        }
                                      }
 -                                    catch (GridCacheEntryRemovedException ignored) {
 -                                        assert dhtEntry.obsolete() : " Got removed exception for non-obsolete entry: "
 -                                            + dhtEntry;
  
 -                                        if (log.isDebugEnabled())
 -                                            log.debug("Got removed exception for DHT entry in map (will ignore): "
 -                                                + dhtEntry);
 -                                    }
 -                                }
 +                                    GridCacheVersion dhtVer = null;
  
 -                                GridCacheVersion dhtVer = null;
 +                                    if (val != null) {
 +                                        dhtVer = val.get1();
  
 -                                if (val != null) {
 -                                    dhtVer = val.get1();
 +                                        valMap.put(
 +                                            key,
 +                                            val);
 +                                    }
  
 -                                    valMap.put(key, val);
 -                                }
 +                                    if (!cand.reentry()) {
 +                                        if (req == null) {
 +                                            boolean clientFirst = false;
  
 -                                if (!cand.reentry()) {
 -                                    if (req == null) {
 -                                        boolean clientFirst = false;
 +                                            if (first) {
 +                                                clientFirst = clientNode &&
 +                                                    !topLocked &&
 +                                                    (tx == null || !tx.hasRemoteLocks());
  
 -                                        if (first) {
 -                                            clientFirst = clientNode &&
 -                                                !topLocked &&
 -                                                (tx == null || !tx.hasRemoteLocks());
 +                                                first = false;
 +                                            }
  
 -                                            first = false;
 +                                            req = new GridNearLockRequest(
 +                                                cctx.cacheId(),
 +                                                topVer,
 +                                                cctx.nodeId(),
 +                                                threadId,
 +                                                futId,
 +                                                lockVer,
 +                                                inTx(),
 +                                                implicitTx(),
 +                                                implicitSingleTx(),
 +                                                read,
 +                                                retval,
 +                                                isolation(),
 +                                                isInvalidate(),
 +                                                timeout,
 +                                                mappedKeys.size(),
 +                                                inTx() ? tx.size() : mappedKeys.size(),
 +                                                inTx() && tx.syncCommit(),
 +                                                inTx() ? tx.subjectId() : null,
 +                                                inTx() ? tx.taskNameHash() : 0,
 +                                                read ? accessTtl : -1L,
 +                                                skipStore,
++                                                keepBinary,
 +                                                clientFirst,
 +                                                cctx.deploymentEnabled());
 +
 +                                            mapping.request(req);
                                          }
  
 -                                        req = new GridNearLockRequest(
 -                                            cctx.cacheId(),
 -                                            topVer,
 -                                            cctx.nodeId(),
 -                                            threadId,
 -                                            futId,
 -                                            lockVer,
 -                                            inTx(),
 -                                            implicitTx(),
 -                                            implicitSingleTx(),
 -                                            read,
 -                                            retval,
 -                                            isolation(),
 -                                            isInvalidate(),
 -                                            timeout,
 -                                            mappedKeys.size(),
 -                                            inTx() ? tx.size() : mappedKeys.size(),
 -                                            inTx() && tx.syncCommit(),
 -                                            inTx() ? tx.subjectId() : null,
 -                                            inTx() ? tx.taskNameHash() : 0,
 -                                            read ? accessTtl : -1L,
 -                                            skipStore,
 -                                            keepBinary,
 -                                            clientFirst,
 -                                            cctx.deploymentEnabled());
 -
 -                                        mapping.request(req);
 -                                    }
 +                                        distributedKeys.add(key);
  
 -                                    distributedKeys.add(key);
 +                                        if (tx != null)
 +                                            tx.addKeyMapping(
 +                                                txKey,
 +                                                mapping.node());
  
 -                                    if (tx != null)
 -                                        tx.addKeyMapping(txKey, mapping.node());
 +                                        req.addKeyBytes(
 +                                            key,
 +                                            retval && dhtVer == null,
 +                                            dhtVer,
 +                                            // Include DHT version to match remote DHT entry.
 +                                            cctx);
 +                                    }
  
 -                                    req.addKeyBytes(
 -                                        key,
 -                                        retval && dhtVer == null,
 -                                        dhtVer, // Include DHT version to match remote DHT entry.
 -                                        cctx);
 +                                    if (cand.reentry())
 +                                        explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
                                  }
 -
 -                                if (cand.reentry())
 +                                else
 +                                    // Ignore reentries within transactions.
                                      explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
 -                            }
 -                            else
 -                                // Ignore reentries within transactions.
 -                                explicit = tx != null && !entry.hasLockCandidate(tx.xidVersion());
  
 -                            if (explicit)
 -                                tx.addKeyMapping(txKey, mapping.node());
 +                                if (explicit)
 +                                    tx.addKeyMapping(
 +                                        txKey,
 +                                        mapping.node());
  
 -                            break;
 -                        }
 -                        catch (GridCacheEntryRemovedException ignored) {
 -                            assert entry.obsolete() : "Got removed exception on non-obsolete entry: " + entry;
 +                                break;
 +                            }
 +                            catch (GridCacheEntryRemovedException ignored) {
 +                                assert entry.obsolete() : "Got removed exception on non-obsolete entry: " + entry;
  
 -                            if (log.isDebugEnabled())
 -                                log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
 +                                if (log.isDebugEnabled())
 +                                    log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
 +                            }
                          }
 -                    }
  
 -                    // Mark mapping explicit lock flag.
 -                    if (explicit) {
 -                        boolean marked = tx != null && tx.markExplicit(node.id());
 +                        // Mark mapping explicit lock flag.
 +                        if (explicit) {
 +                            boolean marked = tx != null && tx.markExplicit(node.id());
  
 -                        assert tx == null || marked;
 +                            assert tx == null || marked;
 +                        }
                      }
 -                }
  
 -                if (!distributedKeys.isEmpty())
 -                    mapping.distributedKeys(distributedKeys);
 -                else {
 -                    assert mapping.request() == null;
 +                    if (!distributedKeys.isEmpty())
 +                        mapping.distributedKeys(distributedKeys);
 +                    else {
 +                        assert mapping.request() == null;
  
 -                    iter.remove();
 +                        iter.remove();
 +                    }
                  }
              }
  
@@@ -1454,165 -1440,158 +1461,166 @@@
           * @param res Result callback.
           */
          void onResult(GridNearLockResponse res) {
 -            if (rcvRes.compareAndSet(false, true)) {
 -                if (res.error() != null) {
 -                    if (log.isDebugEnabled())
 -                        log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
 -                            ", res=" + res + ']');
 +            synchronized (this) {
 +                if (!rcvRes)
 +                    rcvRes = true;
 +                else
 +                    return;
 +            }
  
 -                    // Fail.
 -                    if (res.error() instanceof GridCacheLockTimeoutException)
 -                        onDone(false);
 -                    else
 -                        onDone(res.error());
 +            if (res.error() != null) {
 +                if (log.isDebugEnabled())
 +                    log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
 +                        ", res=" + res + ']');
  
 -                    return;
 -                }
 +                // Fail.
 +                if (res.error() instanceof GridCacheLockTimeoutException)
 +                    onDone(false);
 +                else
 +                    onDone(res.error());
  
 -                if (res.clientRemapVersion() != null) {
 -                    assert cctx.kernalContext().clientNode();
 +                return;
 +            }
  
 -                    IgniteInternalFuture<?> affFut =
 -                        cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
 +            if (res.clientRemapVersion() != null) {
 +                assert cctx.kernalContext().clientNode();
  
 -                    if (affFut != null && !affFut.isDone()) {
 -                        affFut.listen(new CI1<IgniteInternalFuture<?>>() {
 -                            @Override public void apply(IgniteInternalFuture<?> fut) {
 -                                try {
 -                                    fut.get();
 +                IgniteInternalFuture<?> affFut =
 +                    cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
  
 -                                    remap();
 -                                }
 -                                catch (IgniteCheckedException e) {
 -                                    onDone(e);
 -                                }
 -                                finally {
 -                                    cctx.shared().txContextReset();
 -                                }
 +                if (affFut != null && !affFut.isDone()) {
 +                    affFut.listen(new CI1<IgniteInternalFuture<?>>() {
 +                        @Override public void apply(IgniteInternalFuture<?> fut) {
 +                            try {
 +                                fut.get();
 +
 +                                remap();
                              }
 -                        });
 -                    }
 -                    else
 -                        remap();
 +                            catch (IgniteCheckedException e) {
 +                                onDone(e);
 +                            }
 +                            finally {
 +                                cctx.shared().txContextReset();
 +                            }
 +                        }
 +                    });
                  }
 -                else {
 -                    int i = 0;
 +                else
 +                    remap();
 +            }
 +            else {
 +                int i = 0;
  
 -                    AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer.get();
 +                AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer;
  
 -                    for (KeyCacheObject k : keys) {
 -                        while (true) {
 -                            GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
 +                for (KeyCacheObject k : keys) {
 +                    while (true) {
 +                        GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
  
 -                            try {
 -                                if (res.dhtVersion(i) == null) {
 -                                    onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
 -                                        "(will fail the lock): " + res));
 +                        try {
 +                            if (res.dhtVersion(i) == null) {
 +                                onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
 +                                    "(will fail the lock): " + res));
  
 -                                    return;
 -                                }
 +                                return;
 +                            }
  
 -                                IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
 +                            IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
  
 -                                CacheObject oldVal = entry.rawGet();
 -                                boolean hasOldVal = false;
 -                                CacheObject newVal = res.value(i);
 +                            CacheObject oldVal = entry.rawGet();
 +                            boolean hasOldVal = false;
 +                            CacheObject newVal = res.value(i);
  
 -                                boolean readRecordable = false;
 +                            boolean readRecordable = false;
  
 -                                if (retval) {
 -                                    readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
 +                            if (retval) {
 +                                readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
  
 -                                    if (readRecordable)
 -                                        hasOldVal = entry.hasValue();
 -                                }
 +                                if (readRecordable)
 +                                    hasOldVal = entry.hasValue();
 +                            }
  
 -                                GridCacheVersion dhtVer = res.dhtVersion(i);
 -                                GridCacheVersion mappedVer = res.mappedVersion(i);
 +                            GridCacheVersion dhtVer = res.dhtVersion(i);
 +                            GridCacheVersion mappedVer = res.mappedVersion(i);
  
 -                                if (newVal == null) {
 -                                    if (oldValTup != null) {
 -                                        if (oldValTup.get1().equals(dhtVer))
 -                                            newVal = oldValTup.get2();
 +                            if (newVal == null) {
 +                                if (oldValTup != null) {
 +                                    if (oldValTup.get1().equals(dhtVer))
 +                                        newVal = oldValTup.get2();
  
 -                                        oldVal = oldValTup.get2();
 -                                    }
 +                                    oldVal = oldValTup.get2();
                                  }
 +                            }
  
 -                                // Lock is held at this point, so we can set the
 -                                // returned value if any.
 -                                entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
 +                            // Lock is held at this point, so we can set the
 +                            // returned value if any.
 +                            entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
  
 -                                if (inTx()) {
 -                                    tx.hasRemoteLocks(true);
 +                            if (inTx()) {
 +                                tx.hasRemoteLocks(true);
  
 -                                    if (implicitTx() && tx.onePhaseCommit()) {
 -                                        boolean pass = res.filterResult(i);
 +                                if (implicitTx() && tx.onePhaseCommit()) {
 +                                    boolean pass = res.filterResult(i);
  
 -                                        tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
 -                                    }
 +                                    tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
                                  }
 +                            }
  
 -                                entry.readyNearLock(lockVer,
 -                                    mappedVer,
 -                                    res.committedVersions(),
 -                                    res.rolledbackVersions(),
 -                                    res.pending());
 -
 -                                if (retval) {
 -                                    if (readRecordable)
 -                                        cctx.events().addEvent(
 -                                            entry.partition(),
 -                                            entry.key(),
 -                                            tx,
 -                                            null,
 -                                            EVT_CACHE_OBJECT_READ,
 -                                            newVal,
 -                                            newVal != null,
 -                                            oldVal,
 -                                            hasOldVal,
 -                                            CU.subjectId(tx, cctx.shared()),
 -                                            null,
 -                                            inTx() ? tx.resolveTaskName() : null,
 -                                            keepBinary);
 -
 -                                    if (cctx.cache().configuration().isStatisticsEnabled())
 -                                        cctx.cache().metrics0().onRead(false);
 -                                }
 +                            entry.readyNearLock(lockVer,
 +                                mappedVer,
 +                                res.committedVersions(),
 +                                res.rolledbackVersions(),
 +                                res.pending());
 +
 +                            if (retval) {
 +                                if (readRecordable)
 +                                    cctx.events().addEvent(
 +                                        entry.partition(),
 +                                        entry.key(),
 +                                        tx,
 +                                        null,
 +                                        EVT_CACHE_OBJECT_READ,
 +                                        newVal,
 +                                        newVal != null,
 +                                        oldVal,
 +                                        hasOldVal,
 +                                        CU.subjectId(tx, cctx.shared()),
 +                                        null,
-                                         inTx() ? tx.resolveTaskName() : null);
++                                        inTx() ? tx.resolveTaskName() : null,
++                                        keepBinary);
 +
 +                                if (cctx.cache().configuration().isStatisticsEnabled())
 +                                    cctx.cache().metrics0().onRead(false);
 +                            }
  
 -                                if (log.isDebugEnabled())
 -                                    log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
 +                            if (log.isDebugEnabled())
 +                                log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
  
 -                                break; // Inner while loop.
 -                            }
 -                            catch (GridCacheEntryRemovedException ignored) {
 -                                if (log.isDebugEnabled())
 -                                    log.debug("Failed to add candidates because entry was removed (will renew).");
 +                            break; // Inner while loop.
 +                        }
 +                        catch (GridCacheEntryRemovedException ignored) {
 +                            if (log.isDebugEnabled())
 +                                log.debug("Failed to add candidates because entry was removed (will renew).");
  
 +                            synchronized (GridNearLockFuture.this) {
                                  // Replace old entry with new one.
 -                                entries.set(i, (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
 +                                entries.set(i,
 +                                    (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
                              }
                          }
 -
 -                        i++;
                      }
  
 -                    try {
 -                        proceedMapping(mappings);
 -                    }
 -                    catch (IgniteCheckedException e) {
 -                        onDone(e);
 -                    }
 +                    i++;
 +                }
  
 -                    onDone(true);
 +                try {
 +                    proceedMapping();
 +                }
 +                catch (IgniteCheckedException e) {
 +                    onDone(e);
                  }
 +
 +                onDone(true);
              }
          }
  


[2/8] ignite git commit: debugging slowdowns

Posted by ag...@apache.org.
debugging slowdowns


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

Branch: refs/heads/ignite-1.5
Commit: 5e6d0ffefb7d58cbc21bea651671d4de02abf622
Parents: 8e7e330
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Nov 20 19:03:40 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Nov 20 19:03:40 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtGetFuture.java |  24 +-
 .../distributed/dht/GridDhtLockFuture.java      |  78 +-
 .../colocated/GridDhtColocatedLockFuture.java   | 618 ++++++++--------
 .../distributed/near/GridNearGetFuture.java     |   2 -
 .../distributed/near/GridNearLockFuture.java    | 719 ++++++++++---------
 .../distributed/near/GridNearLockMapping.java   |   6 +-
 6 files changed, 748 insertions(+), 699 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 7108da6..6b696b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
@@ -37,7 +38,6 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
@@ -83,7 +83,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private LinkedHashMap<KeyCacheObject, Boolean> keys;
 
     /** Reserved partitions. */
-    private Collection<GridDhtLocalPartition> parts = new GridLeanSet<>(5);
+    private Collection<GridDhtLocalPartition> parts = new HashSet<>();
 
     /** Future ID. */
     private IgniteUuid futId;
@@ -98,7 +98,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private IgniteTxLocalEx tx;
 
     /** Retries because ownership changed. */
-    private Collection<Integer> retries = new GridLeanSet<>();
+    private Collection<Integer> retries;
 
     /** Subject ID. */
     private UUID subjId;
@@ -174,7 +174,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
 
     /** {@inheritDoc} */
     @Override public Collection<Integer> invalidPartitions() {
-        return retries;
+        return retries == null ? Collections.<Integer>emptyList() : retries;
     }
 
     /**
@@ -210,8 +210,12 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private void map(final LinkedHashMap<KeyCacheObject, Boolean> keys) {
         GridDhtFuture<Object> fut = cctx.dht().dhtPreloader().request(keys.keySet(), topVer);
 
-        if (!F.isEmpty(fut.invalidPartitions()))
+        if (!F.isEmpty(fut.invalidPartitions())) {
+            if (retries == null)
+                retries = new HashSet<>();
+
             retries.addAll(fut.invalidPartitions());
+        }
 
         add(new GridEmbeddedFuture<>(
             new IgniteBiClosure<Object, Exception, Collection<GridCacheEntryInfo>>() {
@@ -229,9 +233,13 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     for (Map.Entry<KeyCacheObject, Boolean> key : keys.entrySet()) {
                         int part = cctx.affinity().partition(key.getKey());
 
-                        if (!retries.contains(part)) {
-                            if (!map(key.getKey(), parts))
+                        if (retries == null || !retries.contains(part)) {
+                            if (!map(key.getKey(), parts)) {
+                                if (retries == null)
+                                    retries = new HashSet<>();
+
                                 retries.add(part);
+                            }
                             else
                                 mappedKeys.put(key.getKey(), key.getValue());
                         }
@@ -441,4 +449,4 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private GridDhtCacheAdapter<K, V> cache() {
         return (GridDhtCacheAdapter<K, V>)cctx.cache();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index a7978c9..543acb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -19,13 +19,14 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
@@ -51,8 +52,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
-import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -63,7 +62,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.NotNull;
@@ -123,7 +121,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     private boolean read;
 
     /** Error. */
-    private AtomicReference<Throwable> err = new AtomicReference<>(null);
+    private Throwable err;
 
     /** Timed out flag. */
     private volatile boolean timedOut;
@@ -142,19 +140,16 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     private GridDhtTxLocalAdapter tx;
 
     /** All replies flag. */
-    private AtomicBoolean mapped = new AtomicBoolean(false);
+    private boolean mapped;
 
     /** */
-    private Collection<Integer> invalidParts = new GridLeanSet<>();
+    private Collection<Integer> invalidParts;
 
     /** Trackable flag. */
     private boolean trackable = true;
 
-    /** Mutex. */
-    private final Object mux = new Object();
-
     /** Pending locks. */
-    private final Collection<KeyCacheObject> pendingLocks = new GridConcurrentHashSet<>();
+    private final Collection<KeyCacheObject> pendingLocks;
 
     /** TTL for read operation. */
     private long accessTtl;
@@ -231,6 +226,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         futId = IgniteUuid.randomUuid();
 
         entries = new ArrayList<>(cnt);
+        pendingLocks = U.newHashSet(cnt);
 
         if (log == null)
             log = U.logger(cctx.kernalContext(), logRef, GridDhtLockFuture.class);
@@ -244,7 +240,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
 
     /** {@inheritDoc} */
     @Override public Collection<Integer> invalidPartitions() {
-        return invalidParts;
+        return invalidParts == null ? Collections.<Integer>emptyList() : invalidParts;
     }
 
     /**
@@ -252,6 +248,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      * @param invalidPart Partition to retry.
      */
     void addInvalidPartition(GridCacheContext<?, ?> cacheCtx, int invalidPart) {
+        if (invalidParts == null)
+            invalidParts = new HashSet<>();
+
         invalidParts.add(invalidPart);
 
         // Register invalid partitions with transaction.
@@ -287,10 +286,8 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /**
      * @return Entries.
      */
-    public Collection<GridDhtCacheEntry> entriesCopy() {
-        synchronized (mux) {
-            return new ArrayList<>(entries());
-        }
+    public synchronized Collection<GridDhtCacheEntry> entriesCopy() {
+        return new ArrayList<>(entries());
     }
 
     /**
@@ -403,12 +400,12 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             return null;
         }
 
-        synchronized (mux) {
+        synchronized (this) {
             entries.add(c == null || c.reentry() ? null : entry);
-        }
 
-        if (c != null && !c.reentry())
-            pendingLocks.add(entry.key());
+            if (c != null && !c.reentry())
+                pendingLocks.add(entry.key());
+        }
 
         // Double check if the future has already timed out.
         if (timedOut) {
@@ -615,19 +612,17 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     }
 
     /**
-     * @param e Error.
-     */
-    public void onError(GridDistributedLockCancelledException e) {
-        if (err.compareAndSet(null, e))
-            onComplete(false);
-    }
-
-    /**
      * @param t Error.
      */
     public void onError(Throwable t) {
-        if (err.compareAndSet(null, t))
-            onComplete(false);
+        synchronized (this) {
+            if (err != null)
+                return;
+
+            err = t;
+        }
+
+        onComplete(false);
     }
 
     /**
@@ -667,7 +662,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             log.debug("Received onOwnerChanged() callback [entry=" + entry + ", owner=" + owner + "]");
 
         if (owner != null && owner.version().equals(lockVer)) {
-            pendingLocks.remove(entry.key());
+            synchronized (this) {
+                pendingLocks.remove(entry.key());
+            }
 
             if (checkLocks())
                 map(entries());
@@ -681,7 +678,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /**
      * @return {@code True} if locks have been acquired.
      */
-    private boolean checkLocks() {
+    private synchronized boolean checkLocks() {
         return pendingLocks.isEmpty();
     }
 
@@ -713,7 +710,10 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         if (isDone() || (err == null && success && !checkLocks()))
             return false;
 
-        this.err.compareAndSet(null, err);
+        synchronized (this) {
+            if (this.err == null)
+                this.err = err;
+        }
 
         return onComplete(success);
     }
@@ -734,10 +734,10 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         if (tx != null)
             cctx.tm().txContext(tx);
 
-        if (err.get() == null)
+        if (err == null)
             loadMissingFromStore();
 
-        if (super.onDone(success, err.get())) {
+        if (super.onDone(success, err)) {
             if (log.isDebugEnabled())
                 log.debug("Completing future: " + this);
 
@@ -778,11 +778,11 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      * @param entries Entries.
      */
     private void map(Iterable<GridDhtCacheEntry> entries) {
-        if (!mapped.compareAndSet(false, true)) {
-            if (log.isDebugEnabled())
-                log.debug("Will not map DHT lock future (other thread is mapping): " + this);
+        synchronized (this) {
+            if (mapped)
+                return;
 
-            return;
+            mapped = true;
         }
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 8245d88..7e17efe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Deque;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
@@ -66,12 +66,10 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
 
@@ -113,7 +111,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     private boolean retval;
 
     /** Error. */
-    private AtomicReference<Throwable> err = new AtomicReference<>(null);
+    private volatile Throwable err;
 
     /** Timeout object. */
     @GridToStringExclude
@@ -130,7 +128,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     private GridNearTxLocal tx;
 
     /** Topology snapshot to operate on. */
-    private AtomicReference<AffinityTopologyVersion> topVer = new AtomicReference<>();
+    private volatile AffinityTopologyVersion topVer;
 
     /** Map of current values. */
     private Map<KeyCacheObject, IgniteBiTuple<GridCacheVersion, CacheObject>> valMap;
@@ -144,6 +142,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     /** Skip store flag. */
     private final boolean skipStore;
 
+    /** */
+    private Deque<GridNearLockMapping> mappings;
+
     /**
      * @param cctx Registry.
      * @param keys Keys to lock.
@@ -196,7 +197,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             cctx.time().addTimeoutObject(timeoutObj);
         }
 
-        valMap = new ConcurrentHashMap8<>(keys.size(), 1f);
+        valMap = new ConcurrentHashMap8<>();
     }
 
     /** {@inheritDoc} */
@@ -318,7 +319,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     false,
                     null);
 
-                cand.topologyVersion(topVer.get());
+                cand.topologyVersion(topVer);
             }
         }
         else {
@@ -338,12 +339,12 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     false,
                     null);
 
-                cand.topologyVersion(topVer.get());
+                cand.topologyVersion(topVer);
             }
             else
                 cand = cand.reenter();
 
-            cctx.mvcc().addExplicitLock(threadId, cand, topVer.get());
+            cctx.mvcc().addExplicitLock(threadId, cand, topVer);
         }
 
         return cand;
@@ -479,8 +480,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     /**
      * @param t Error.
      */
-    private void onError(Throwable t) {
-        err.compareAndSet(null, t instanceof GridCacheLockTimeoutException ? null : t);
+    private synchronized void onError(Throwable t) {
+        if (err == null && !(t instanceof GridCacheLockTimeoutException))
+            err = t;
     }
 
     /** {@inheritDoc} */
@@ -499,7 +501,8 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         if (isDone())
             return false;
 
-        this.err.compareAndSet(null, err instanceof GridCacheLockTimeoutException ? null : err);
+        if (err != null)
+            onError(err);
 
         if (err != null)
             success = false;
@@ -525,7 +528,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         if (tx != null)
             cctx.tm().txContext(tx);
 
-        if (super.onDone(success, err.get())) {
+        if (super.onDone(success, err)) {
             if (log.isDebugEnabled())
                 log.debug("Completing future: " + this);
 
@@ -617,7 +620,10 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             }
 
             // Continue mapping on the same topology version as it was before.
-            this.topVer.compareAndSet(null, topVer);
+            synchronized (this) {
+                if (this.topVer == null)
+                    this.topVer = topVer;
+            }
 
             map(keys, false, true);
 
@@ -666,13 +672,18 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     if (tx != null)
                         tx.onRemap(topVer);
 
-                    this.topVer.set(topVer);
+                    synchronized (this) {
+                        this.topVer = topVer;
+                    }
                 }
                 else {
                     if (tx != null)
                         tx.topologyVersion(topVer);
 
-                    this.topVer.compareAndSet(null, topVer);
+                    synchronized (this) {
+                        if (this.topVer == null)
+                            this.topVer = topVer;
+                    }
                 }
 
                 map(keys, remap, false);
@@ -716,242 +727,256 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      */
     private void map(Collection<KeyCacheObject> keys, boolean remap, boolean topLocked) {
         try {
-            AffinityTopologyVersion topVer = this.topVer.get();
+            map0(
+                keys,
+                remap,
+                topLocked);
+        }
+        catch (IgniteCheckedException ex) {
+            onDone(false, ex);
+        }
+    }
 
-            assert topVer != null;
+    private synchronized void map0(
+        Collection<KeyCacheObject> keys,
+        boolean remap,
+        boolean topLocked
+    ) throws IgniteCheckedException {
+        AffinityTopologyVersion topVer = this.topVer;
 
-            assert topVer.topologyVersion() > 0;
+        assert topVer != null;
 
-            if (CU.affinityNodes(cctx, topVer).isEmpty()) {
-                onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
-                    "(all partition nodes left the grid): " + cctx.name()));
+        assert topVer.topologyVersion() > 0;
 
-                return;
-            }
+        if (CU.affinityNodes(cctx, topVer).isEmpty()) {
+            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for cache " +
+                "(all partition nodes left the grid): " + cctx.name()));
 
-            boolean clientNode = cctx.kernalContext().clientNode();
+            return;
+        }
 
-            assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
+        boolean clientNode = cctx.kernalContext().clientNode();
 
-            // First assume this node is primary for all keys passed in.
-            if (!clientNode && mapAsPrimary(keys, topVer))
-                return;
+        assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
 
-            Deque<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
+        // First assume this node is primary for all keys passed in.
+        if (!clientNode && mapAsPrimary(keys, topVer))
+            return;
 
-            // Assign keys to primary nodes.
-            GridNearLockMapping map = null;
+        mappings = new ArrayDeque<>();
 
-            for (KeyCacheObject key : keys) {
-                GridNearLockMapping updated = map(key, map, topVer);
+        // Assign keys to primary nodes.
+        GridNearLockMapping map = null;
 
-                // If new mapping was created, add to collection.
-                if (updated != map) {
-                    mappings.add(updated);
+        for (KeyCacheObject key : keys) {
+            GridNearLockMapping updated = map(key, map, topVer);
 
-                    if (tx != null && updated.node().isLocal())
-                        tx.colocatedLocallyMapped(true);
-                }
+            // If new mapping was created, add to collection.
+            if (updated != map) {
+                mappings.add(updated);
 
-                map = updated;
+                if (tx != null && updated.node().isLocal())
+                    tx.colocatedLocallyMapped(true);
             }
 
-            if (isDone()) {
-                if (log.isDebugEnabled())
-                    log.debug("Abandoning (re)map because future is done: " + this);
-
-                return;
-            }
+            map = updated;
+        }
 
+        if (isDone()) {
             if (log.isDebugEnabled())
-                log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
+                log.debug("Abandoning (re)map because future is done: " + this);
+
+            return;
+        }
 
-            boolean hasRmtNodes = false;
+        if (log.isDebugEnabled())
+            log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
 
-            boolean first = true;
+        boolean hasRmtNodes = false;
 
-            // Create mini futures.
-            for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
-                GridNearLockMapping mapping = iter.next();
+        boolean first = true;
 
-                ClusterNode node = mapping.node();
-                Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
+        // Create mini futures.
+        for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
+            GridNearLockMapping mapping = iter.next();
 
-                boolean loc = node.equals(cctx.localNode());
+            ClusterNode node = mapping.node();
+            Collection<KeyCacheObject> mappedKeys = mapping.mappedKeys();
 
-                assert !mappedKeys.isEmpty();
+            boolean loc = node.equals(cctx.localNode());
 
-                GridNearLockRequest req = null;
+            assert !mappedKeys.isEmpty();
 
-                Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
+            GridNearLockRequest req = null;
 
-                for (KeyCacheObject key : mappedKeys) {
-                    IgniteTxKey txKey = cctx.txKey(key);
+            Collection<KeyCacheObject> distributedKeys = new ArrayList<>(mappedKeys.size());
 
-                    GridDistributedCacheEntry entry = null;
+            for (KeyCacheObject key : mappedKeys) {
+                IgniteTxKey txKey = cctx.txKey(key);
 
-                    if (tx != null) {
-                        IgniteTxEntry txEntry = tx.entry(txKey);
+                GridDistributedCacheEntry entry = null;
 
-                        if (txEntry != null) {
-                            entry = (GridDistributedCacheEntry)txEntry.cached();
+                if (tx != null) {
+                    IgniteTxEntry txEntry = tx.entry(txKey);
 
-                            if (entry != null && !(loc ^ entry.detached())) {
-                                entry = cctx.colocated().entryExx(key, topVer, true);
+                    if (txEntry != null) {
+                        entry = (GridDistributedCacheEntry)txEntry.cached();
 
-                                txEntry.cached(entry);
-                            }
+                        if (entry != null && !(loc ^ entry.detached())) {
+                            entry = cctx.colocated().entryExx(key, topVer, true);
+
+                            txEntry.cached(entry);
                         }
                     }
+                }
 
-                    boolean explicit;
+                boolean explicit;
 
-                    while (true) {
-                        try {
-                            if (entry == null)
-                                entry = cctx.colocated().entryExx(key, topVer, true);
+                while (true) {
+                    try {
+                        if (entry == null)
+                            entry = cctx.colocated().entryExx(key, topVer, true);
 
-                            if (!cctx.isAll(entry, filter)) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Entry being locked did not pass filter (will not lock): " + entry);
+                        if (!cctx.isAll(entry, filter)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Entry being locked did not pass filter (will not lock): " + entry);
 
-                                onComplete(false, false);
+                            onComplete(false, false);
 
-                                return;
-                            }
+                            return;
+                        }
 
-                            assert loc ^ entry.detached() : "Invalid entry [loc=" + loc + ", entry=" + entry + ']';
+                        assert loc ^ entry.detached() : "Invalid entry [loc=" + loc + ", entry=" + entry + ']';
 
-                            GridCacheMvccCandidate cand = addEntry(entry);
+                        GridCacheMvccCandidate cand = addEntry(entry);
 
-                            // Will either return value from dht cache or null if this is a miss.
-                            IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.detached() ? null :
-                                ((GridDhtCacheEntry)entry).versionedValue(topVer);
+                        // Will either return value from dht cache or null if this is a miss.
+                        IgniteBiTuple<GridCacheVersion, CacheObject> val = entry.detached() ? null :
+                            ((GridDhtCacheEntry)entry).versionedValue(topVer);
 
-                            GridCacheVersion dhtVer = null;
+                        GridCacheVersion dhtVer = null;
 
-                            if (val != null) {
-                                dhtVer = val.get1();
+                        if (val != null) {
+                            dhtVer = val.get1();
 
-                                valMap.put(key, val);
-                            }
+                            valMap.put(key, val);
+                        }
 
-                            if (cand != null && !cand.reentry()) {
-                                if (req == null) {
-                                    boolean clientFirst = false;
-
-                                    if (first) {
-                                        clientFirst = clientNode &&
-                                            !topLocked &&
-                                            (tx == null || !tx.hasRemoteLocks());
-
-                                        first = false;
-                                    }
-
-                                    req = new GridNearLockRequest(
-                                        cctx.cacheId(),
-                                        topVer,
-                                        cctx.nodeId(),
-                                        threadId,
-                                        futId,
-                                        lockVer,
-                                        inTx(),
-                                        implicitTx(),
-                                        implicitSingleTx(),
-                                        read,
-                                        retval,
-                                        isolation(),
-                                        isInvalidate(),
-                                        timeout,
-                                        mappedKeys.size(),
-                                        inTx() ? tx.size() : mappedKeys.size(),
-                                        inTx() && tx.syncCommit(),
-                                        inTx() ? tx.subjectId() : null,
-                                        inTx() ? tx.taskNameHash() : 0,
-                                        read ? accessTtl : -1L,
-                                        skipStore,
-                                        clientFirst,
-                                        cctx.deploymentEnabled());
-
-                                    mapping.request(req);
-                                }
+                        if (cand != null && !cand.reentry()) {
+                            if (req == null) {
+                                boolean clientFirst = false;
 
-                                distributedKeys.add(key);
+                                if (first) {
+                                    clientFirst = clientNode &&
+                                        !topLocked &&
+                                        (tx == null || !tx.hasRemoteLocks());
 
-                                if (tx != null)
-                                    tx.addKeyMapping(txKey, mapping.node());
+                                    first = false;
+                                }
 
-                                req.addKeyBytes(
-                                    key,
+                                req = new GridNearLockRequest(
+                                    cctx.cacheId(),
+                                    topVer,
+                                    cctx.nodeId(),
+                                    threadId,
+                                    futId,
+                                    lockVer,
+                                    inTx(),
+                                    implicitTx(),
+                                    implicitSingleTx(),
+                                    read,
                                     retval,
-                                    dhtVer, // Include DHT version to match remote DHT entry.
-                                    cctx);
+                                    isolation(),
+                                    isInvalidate(),
+                                    timeout,
+                                    mappedKeys.size(),
+                                    inTx() ? tx.size() : mappedKeys.size(),
+                                    inTx() && tx.syncCommit(),
+                                    inTx() ? tx.subjectId() : null,
+                                    inTx() ? tx.taskNameHash() : 0,
+                                    read ? accessTtl : -1L,
+                                    skipStore,
+                                    clientFirst,
+                                    cctx.deploymentEnabled());
+
+                                mapping.request(req);
                             }
 
-                            explicit = inTx() && cand == null;
+                            distributedKeys.add(key);
 
-                            if (explicit)
+                            if (tx != null)
                                 tx.addKeyMapping(txKey, mapping.node());
 
-                            break;
+                            req.addKeyBytes(
+                                key,
+                                retval,
+                                dhtVer, // Include DHT version to match remote DHT entry.
+                                cctx);
                         }
-                        catch (GridCacheEntryRemovedException ignored) {
-                            if (log.isDebugEnabled())
-                                log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
 
-                            entry = null;
-                        }
-                    }
+                        explicit = inTx() && cand == null;
 
-                    // Mark mapping explicit lock flag.
-                    if (explicit) {
-                        boolean marked = tx != null && tx.markExplicit(node.id());
+                        if (explicit)
+                            tx.addKeyMapping(txKey, mapping.node());
 
-                        assert tx == null || marked;
+                        break;
                     }
-                }
-
-                if (inTx() && req != null)
-                    req.hasTransforms(tx.hasTransforms());
+                    catch (GridCacheEntryRemovedException ignored) {
+                        if (log.isDebugEnabled())
+                            log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
 
-                if (!distributedKeys.isEmpty()) {
-                    mapping.distributedKeys(distributedKeys);
-
-                    hasRmtNodes |= !mapping.node().isLocal();
+                        entry = null;
+                    }
                 }
-                else {
-                    assert mapping.request() == null;
 
-                    iter.remove();
+                // Mark mapping explicit lock flag.
+                if (explicit) {
+                    boolean marked = tx != null && tx.markExplicit(node.id());
+
+                    assert tx == null || marked;
                 }
             }
 
-            if (hasRmtNodes) {
-                trackable = true;
+            if (inTx() && req != null)
+                req.hasTransforms(tx.hasTransforms());
 
-                if (!remap && !cctx.mvcc().addFuture(this))
-                    throw new IllegalStateException("Duplicate future ID: " + this);
+            if (!distributedKeys.isEmpty()) {
+                mapping.distributedKeys(distributedKeys);
+
+                hasRmtNodes |= !mapping.node().isLocal();
             }
-            else
-                trackable = false;
+            else {
+                assert mapping.request() == null;
 
-            proceedMapping(mappings);
+                iter.remove();
+            }
         }
-        catch (IgniteCheckedException ex) {
-            onDone(false, ex);
+
+        if (hasRmtNodes) {
+            trackable = true;
+
+            if (!remap && !cctx.mvcc().addFuture(this))
+                throw new IllegalStateException("Duplicate future ID: " + this);
         }
+        else
+            trackable = false;
+
+        proceedMapping();
     }
 
     /**
      * Gets next near lock mapping and either acquires dht locks locally or sends near lock request to
      * remote primary node.
      *
-     * @param mappings Queue of mappings.
      * @throws IgniteCheckedException If mapping can not be completed.
      */
-    private void proceedMapping(final Deque<GridNearLockMapping> mappings)
+    private void proceedMapping()
         throws IgniteCheckedException {
-        GridNearLockMapping map = mappings.poll();
+        GridNearLockMapping map;
+
+        synchronized (this) {
+            map = mappings.poll();
+        }
 
         // If there are no more mappings to process, complete the future.
         if (map == null)
@@ -965,9 +990,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             req.filter(filter, cctx);
 
         if (node.isLocal())
-            lockLocally(mappedKeys, req.topologyVersion(), mappings);
+            lockLocally(mappedKeys, req.topologyVersion());
         else {
-            final MiniFuture fut = new MiniFuture(node, mappedKeys, mappings);
+            final MiniFuture fut = new MiniFuture(node, mappedKeys);
 
             req.miniId(fut.futureId());
 
@@ -1016,15 +1041,12 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
 
     /**
      * Locks given keys directly through dht cache.
-     *
-     * @param keys Collection of keys.
+     *  @param keys Collection of keys.
      * @param topVer Topology version to lock on.
-     * @param mappings Optional collection of mappings to proceed locking.
      */
     private void lockLocally(
         final Collection<KeyCacheObject> keys,
-        AffinityTopologyVersion topVer,
-        @Nullable final Deque<GridNearLockMapping> mappings
+        AffinityTopologyVersion topVer
     ) {
         if (log.isDebugEnabled())
             log.debug("Before locally locking keys : " + keys);
@@ -1078,7 +1100,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     try {
                         // Proceed and add new future (if any) before completing embedded future.
                         if (mappings != null)
-                            proceedMapping(mappings);
+                            proceedMapping();
                     }
                     catch (IgniteCheckedException ex) {
                         onError(ex);
@@ -1101,7 +1123,8 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      * @return {@code True} if all keys were mapped locally, {@code false} if full mapping should be performed.
      * @throws IgniteCheckedException If key cannot be added to mapping.
      */
-    private boolean mapAsPrimary(Collection<KeyCacheObject> keys, AffinityTopologyVersion topVer) throws IgniteCheckedException {
+    private boolean mapAsPrimary(Collection<KeyCacheObject> keys, AffinityTopologyVersion topVer)
+        throws IgniteCheckedException {
         // Assign keys to primary nodes.
         Collection<KeyCacheObject> distributedKeys = new ArrayList<>(keys.size());
 
@@ -1137,7 +1160,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     tx.addKeyMapping(cctx.txKey(key), cctx.localNode());
             }
 
-            lockLocally(distributedKeys, topVer, null);
+            lockLocally(distributedKeys, topVer);
         }
 
         return true;
@@ -1221,7 +1244,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         ClusterTopologyCheckedException topEx = new ClusterTopologyCheckedException("Failed to acquire lock for keys " +
             "(primary node left grid, retry transaction if possible) [keys=" + keys + ", node=" + nodeId + ']', nested);
 
-        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer.get()));
+        topEx.retryReadyFuture(cctx.shared().nextAffinityReadyFuture(topVer));
 
         return topEx;
     }
@@ -1275,19 +1298,18 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         private Deque<GridNearLockMapping> mappings;
 
         /** */
-        private AtomicBoolean rcvRes = new AtomicBoolean(false);
+        private boolean rcvRes;
 
         /**
          * @param node Node.
          * @param keys Keys.
-         * @param mappings Mappings to proceed.
          */
-        MiniFuture(ClusterNode node,
-            Collection<KeyCacheObject> keys,
-            Deque<GridNearLockMapping> mappings) {
+        MiniFuture(
+            ClusterNode node,
+            Collection<KeyCacheObject> keys
+        ) {
             this.node = node;
             this.keys = keys;
-            this.mappings = mappings;
         }
 
         /**
@@ -1312,159 +1334,153 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         }
 
         /**
-         * @param e Error.
-         */
-        void onResult(Throwable e) {
-            if (rcvRes.compareAndSet(false, true)) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to get future result [fut=" + this + ", err=" + e + ']');
-
-                // Fail.
-                onDone(e);
-            }
-            else
-                U.warn(log, "Received error after another result has been processed [fut=" +
-                    GridDhtColocatedLockFuture.this + ", mini=" + this + ']', e);
-        }
-
-        /**
          * @param e Node left exception.
          */
         void onResult(ClusterTopologyCheckedException e) {
             if (isDone())
                 return;
 
-            if (rcvRes.compareAndSet(false, true)) {
-                if (log.isDebugEnabled())
-                    log.debug("Remote node left grid while sending or waiting for reply (will fail): " + this);
+            synchronized (this) {
+                if (rcvRes)
+                    return;
+
+                rcvRes = true;
+            }
 
-                if (tx != null)
-                    tx.removeMapping(node.id());
+            if (log.isDebugEnabled())
+                log.debug("Remote node left grid while sending or waiting for reply (will fail): " + this);
 
-                // Primary node left the grid, so fail the future.
-                GridDhtColocatedLockFuture.this.onDone(newTopologyException(e, node.id()));
+            if (tx != null)
+                tx.removeMapping(node.id());
 
-                onDone(true);
-            }
+            // Primary node left the grid, so fail the future.
+            GridDhtColocatedLockFuture.this.onDone(newTopologyException(e, node.id()));
+
+            onDone(true);
         }
 
         /**
          * @param res Result callback.
          */
         void onResult(GridNearLockResponse res) {
-            if (rcvRes.compareAndSet(false, true)) {
-                if (res.error() != null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
-                            ", res=" + res + ']');
-
-                    // Fail.
-                    if (res.error() instanceof GridCacheLockTimeoutException)
-                        onDone(false);
-                    else
-                        onDone(res.error());
-
+            synchronized (this) {
+                if (rcvRes)
                     return;
-                }
 
-                if (res.clientRemapVersion() != null) {
-                    assert cctx.kernalContext().clientNode();
+                rcvRes = true;
+            }
 
-                    IgniteInternalFuture<?> affFut =
-                        cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
+            if (res.error() != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
+                        ", res=" + res + ']');
 
-                    if (affFut != null && !affFut.isDone()) {
-                        affFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                            @Override public void apply(IgniteInternalFuture<?> fut) {
-                                try {
-                                    fut.get();
+                // Fail.
+                if (res.error() instanceof GridCacheLockTimeoutException)
+                    onDone(false);
+                else
+                    onDone(res.error());
 
-                                    remap();
-                                }
-                                catch (IgniteCheckedException e) {
-                                    onDone(e);
-                                }
-                                finally {
-                                    cctx.shared().txContextReset();
-                                }
-                            }
-                        });
-                    }
-                    else
-                        remap();
-                }
-                else  {
-                    int i = 0;
+                return;
+            }
 
-                    for (KeyCacheObject k : keys) {
-                        IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(k);
+            if (res.clientRemapVersion() != null) {
+                assert cctx.kernalContext().clientNode();
 
-                        CacheObject newVal = res.value(i);
+                IgniteInternalFuture<?> affFut =
+                    cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
 
-                        GridCacheVersion dhtVer = res.dhtVersion(i);
+                if (affFut != null && !affFut.isDone()) {
+                    affFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut) {
+                            try {
+                                fut.get();
 
-                        if (newVal == null) {
-                            if (oldValTup != null) {
-                                if (oldValTup.get1().equals(dhtVer))
-                                    newVal = oldValTup.get2();
+                                remap();
+                            }
+                            catch (IgniteCheckedException e) {
+                                onDone(e);
+                            }
+                            finally {
+                                cctx.shared().txContextReset();
                             }
                         }
+                    });
+                }
+                else
+                    remap();
+            }
+            else  {
+                int i = 0;
 
-                        if (inTx()) {
-                            IgniteTxEntry txEntry = tx.entry(cctx.txKey(k));
+                for (KeyCacheObject k : keys) {
+                    IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(k);
 
-                            // In colocated cache we must receive responses only for detached entries.
-                            assert txEntry.cached().detached() : txEntry;
+                    CacheObject newVal = res.value(i);
 
-                            txEntry.markLocked();
+                    GridCacheVersion dhtVer = res.dhtVersion(i);
 
-                            GridDhtDetachedCacheEntry entry = (GridDhtDetachedCacheEntry)txEntry.cached();
+                    if (newVal == null) {
+                        if (oldValTup != null) {
+                            if (oldValTup.get1().equals(dhtVer))
+                                newVal = oldValTup.get2();
+                        }
+                    }
 
-                            if (res.dhtVersion(i) == null) {
-                                onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
-                                    "(will fail the lock): " + res));
+                    if (inTx()) {
+                        IgniteTxEntry txEntry = tx.entry(cctx.txKey(k));
 
-                                return;
-                            }
+                        // In colocated cache we must receive responses only for detached entries.
+                        assert txEntry.cached().detached() : txEntry;
 
-                            // Set value to detached entry.
-                            entry.resetFromPrimary(newVal, dhtVer);
+                        txEntry.markLocked();
 
-                            tx.hasRemoteLocks(true);
+                        GridDhtDetachedCacheEntry entry = (GridDhtDetachedCacheEntry)txEntry.cached();
 
-                            if (log.isDebugEnabled())
-                                log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
-                        }
-                        else
-                            cctx.mvcc().markExplicitOwner(cctx.txKey(k), threadId);
-
-                        if (retval && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
-                            cctx.events().addEvent(cctx.affinity().partition(k),
-                                k,
-                                tx,
-                                null,
-                                EVT_CACHE_OBJECT_READ,
-                                newVal,
-                                newVal != null,
-                                null,
-                                false,
-                                CU.subjectId(tx, cctx.shared()),
-                                null,
-                                tx == null ? null : tx.resolveTaskName());
+                        if (res.dhtVersion(i) == null) {
+                            onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
+                                "(will fail the lock): " + res));
+
+                            return;
                         }
 
-                        i++;
-                    }
+                        // Set value to detached entry.
+                        entry.resetFromPrimary(newVal, dhtVer);
 
-                    try {
-                        proceedMapping(mappings);
+                        tx.hasRemoteLocks(true);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
                     }
-                    catch (IgniteCheckedException e) {
-                        onDone(e);
+                    else
+                        cctx.mvcc().markExplicitOwner(cctx.txKey(k), threadId);
+
+                    if (retval && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
+                        cctx.events().addEvent(cctx.affinity().partition(k),
+                            k,
+                            tx,
+                            null,
+                            EVT_CACHE_OBJECT_READ,
+                            newVal,
+                            newVal != null,
+                            null,
+                            false,
+                            CU.subjectId(tx, cctx.shared()),
+                            null,
+                            tx == null ? null : tx.resolveTaskName());
                     }
 
-                    onDone(true);
+                    i++;
+                }
+
+                try {
+                    proceedMapping();
                 }
+                catch (IgniteCheckedException e) {
+                    onDone(e);
+                }
+
+                onDone(true);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e6d0ffe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index dfaa44e..4a030b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
@@ -60,7 +59,6 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 


[4/8] ignite git commit: Merge branches 'ignite-1.5' and 'ignite-1.5-tx-futs-opts' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5-tx-futs-opts

Posted by ag...@apache.org.
Merge branches 'ignite-1.5' and 'ignite-1.5-tx-futs-opts' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5-tx-futs-opts


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

Branch: refs/heads/ignite-1.5
Commit: 1c8ff109555cdd639ac0c8742093c8faf683b835
Parents: d92de42 457ca6f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Sat Nov 21 17:51:07 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Sat Nov 21 17:51:07 2015 +0300

----------------------------------------------------------------------
 .../internal/direct/DirectByteBufferStream.java | 1499 -----------------
 .../internal/direct/DirectMessageReader.java    |  144 +-
 .../internal/direct/DirectMessageWriter.java    |   65 +-
 .../direct/DirectMessageWriterState.java        |  123 --
 .../direct/state/DirectMessageState.java        |   98 ++
 .../direct/state/DirectMessageStateItem.java    |   28 +
 .../direct/stream/DirectByteBufferStream.java   |  316 ++++
 .../stream/v1/DirectByteBufferStreamImplV1.java | 1360 +++++++++++++++
 .../stream/v2/DirectByteBufferStreamImplV2.java | 1583 ++++++++++++++++++
 .../managers/communication/GridIoManager.java   |   64 +-
 .../processors/cache/GridCacheUtils.java        |   39 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../GridFutureRemapTimeoutObject.java           |   72 -
 .../dht/GridPartitionedGetFuture.java           |   28 +-
 .../distributed/near/GridNearGetFuture.java     |   28 +-
 .../IgniteTxImplicitSingleStateImpl.java        |   29 +-
 .../IgniteTxRemoteSingleStateImpl.java          |   19 +-
 .../datastructures/DataStructuresProcessor.java |   47 +-
 .../GridAtomicCacheQueueImpl.java               |  126 +-
 .../GridCacheAtomicReferenceImpl.java           |   10 +-
 .../GridCacheCountDownLatchImpl.java            |   15 +-
 .../datastructures/GridCacheQueueAdapter.java   |   32 +-
 .../GridTransactionalCacheQueueImpl.java        |  193 +--
 .../ignite/internal/util/IgniteUtils.java       |   31 +
 .../internal/util/ipc/IpcToNioAdapter.java      |   14 +-
 .../util/nio/GridCommunicationClient.java       |    4 +-
 .../internal/util/nio/GridDirectParser.java     |   37 +-
 .../util/nio/GridNioMessageReaderFactory.java   |   37 +
 .../util/nio/GridNioMessageWriterFactory.java   |   35 +
 .../ignite/internal/util/nio/GridNioServer.java |   47 +-
 .../util/nio/GridShmemCommunicationClient.java  |   12 +-
 .../communication/MessageFormatter.java         |   15 +-
 .../extensions/communication/MessageReader.java |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   80 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   36 +
 .../apache/ignite/stream/StreamTransformer.java |    9 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |  924 +++++-----
 ...rtitionedDataStructuresFailoverSelfTest.java |    7 +-
 ...edOffheapDataStructuresFailoverSelfTest.java |   12 +-
 ...eplicatedDataStructuresFailoverSelfTest.java |    5 -
 ...gniteAtomicLongChangingTopologySelfTest.java |    2 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    4 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   90 +-
 .../testframework/GridSpiTestContext.java       |   12 +-
 .../testframework/junits/GridAbstractTest.java  |    6 +-
 .../spring/IgniteTransactionHolder.java         |   97 ++
 .../spring/SpringTransactionManager.java        |  522 ++++++
 .../transactions/spring/package-info.java       |   22 +
 .../test/java/config/spring-transactions.xml    |   36 +
 .../testsuites/IgniteSpringTestSuite.java       |    5 +-
 .../GridSpringTransactionManagerSelfTest.java   |  165 ++
 .../spring/GridSpringTransactionService.java    |   68 +
 parent/pom.xml                                  |    4 +-
 56 files changed, 5585 insertions(+), 2711 deletions(-)
----------------------------------------------------------------------



[8/8] ignite git commit: Merge branch ignite-1.5-tx-futs-opts into ignite-1.5

Posted by ag...@apache.org.
Merge branch ignite-1.5-tx-futs-opts into ignite-1.5


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

Branch: refs/heads/ignite-1.5
Commit: 07f5a62ec8e496a9a43e802f3f7bf5efc1dc2f9a
Parents: 15877a8 388c857
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 09:44:43 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 09:44:43 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtGetFuture.java |  24 +-
 .../distributed/dht/GridDhtLockFuture.java      |  77 +-
 .../colocated/GridDhtColocatedLockFuture.java   | 595 ++++++++-------
 .../distributed/near/GridNearLockFuture.java    | 739 ++++++++++---------
 .../distributed/near/GridNearLockMapping.java   |   6 +-
 5 files changed, 751 insertions(+), 690 deletions(-)
----------------------------------------------------------------------