You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/05/13 16:49:14 UTC

[1/8] incubator-ignite git commit: # ignite-835

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-gg-9614 436227dcd -> 98008911e


# ignite-835


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

Branch: refs/heads/ignite-gg-9614
Commit: ade75f94f1bb63b13cd3cb997cc7820541994858
Parents: 08360c9
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 12 16:55:43 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 12 16:55:43 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTransactionalCacheAdapter.java   | 9 ++++-----
 .../cache/distributed/GridCacheLockAbstractTest.java        | 2 --
 2 files changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ade75f94/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 87026f3..068e8b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -174,7 +174,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                         // Invalidate key in near cache, if any.
                         if (isNearEnabled(cacheCfg))
-                            obsoleteNearEntry(key, req.version());
+                            obsoleteNearEntry(key);
 
                         break;
                     }
@@ -291,7 +291,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                     // Invalidate key in near cache, if any.
                     if (isNearEnabled(cacheCfg))
-                        obsoleteNearEntry(key, req.version());
+                        obsoleteNearEntry(key);
 
                     if (tx != null) {
                         tx.clearEntry(txKey);
@@ -1481,12 +1481,11 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
     /**
      * @param key Key
-     * @param ver Version.
      */
-    private void obsoleteNearEntry(KeyCacheObject key, GridCacheVersion ver) {
+    private void obsoleteNearEntry(KeyCacheObject key) {
         GridCacheEntryEx nearEntry = near().peekEx(key);
 
         if (nearEntry != null)
-            nearEntry.markObsolete(ver);
+            nearEntry.markObsolete(ctx.versions().next());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ade75f94/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
index ab0f7d0..2fe76e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
@@ -501,8 +501,6 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
      * @throws Throwable If failed.
      */
     public void testLockReentrancy() throws Throwable {
-        fail("https://issues.apache.org/jira/browse/IGNITE-835");
-
         Affinity<Integer> aff = ignite1.affinity(null);
 
         for (int i = 10; i < 100; i++) {


[5/8] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-835' into ignite-sprint-5

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-835' into ignite-sprint-5


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

Branch: refs/heads/ignite-gg-9614
Commit: 22341a9ad6e0c7627ffab26ba63c3f8ed3ace9c0
Parents: 5f8925b 96ff8f2
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 13 13:50:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 13 13:50:48 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtTransactionalCacheAdapter.java   |  9 ++--
 .../distributed/GridCacheLockAbstractTest.java  |  2 -
 ...achePartitionedNearDisabledLockSelfTest.java | 47 ++++++++++++++++++++
 .../GridCacheReplicatedLockSelfTest.java        |  5 +++
 .../testsuites/IgniteCacheTestSuite2.java       |  1 +
 5 files changed, 57 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[7/8] incubator-ignite git commit: #ignite-460: Remove valPtr field from GridCacheMapEntry.

Posted by vo...@apache.org.
#ignite-460: Remove valPtr field from GridCacheMapEntry.


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

Branch: refs/heads/ignite-gg-9614
Commit: d87f6d61b343e95f90624fb7bf382c58a1f7ff1e
Parents: 8246788
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 17:45:24 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 17:45:24 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   7 +
 .../processors/cache/GridCacheMapEntry.java     |  70 +++++---
 .../distributed/dht/GridDhtCacheAdapter.java    |  16 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  63 +++++++
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   3 +
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  63 +++++++
 .../dht/colocated/GridDhtColocatedCache.java    |   5 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  63 +++++++
 .../distributed/near/GridNearCacheAdapter.java  |   3 +
 .../near/GridNearOffHeapCacheEntry.java         |  60 +++++++
 .../cache/local/GridLocalCacheEntry.java        |  18 ++
 .../cache/CacheOffheapMapEntrySelfTest.java     | 168 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 14 files changed, 514 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 48dcee7..2eeaed6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1655,6 +1655,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return {@code True} if should use entry with offheap value pointer.
+     */
+    public boolean useOffheapEntry() {
+        return cacheCfg.getMemoryMode() == OFFHEAP_TIERED || cacheCfg.getMemoryMode() == OFFHEAP_VALUES;
+    }
+
+    /**
      * Converts temporary offheap object to heap-based.
      *
      * @param obj Object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 71858d1..86ed57a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -115,9 +115,6 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     @GridToStringInclude
     private final int hash;
 
-    /** Off-heap value pointer. */
-    protected long valPtr;
-
     /** Extras */
     @GridToStringInclude
     private GridCacheEntryExtras extras;
@@ -188,7 +185,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (cctx.cache().isIgfsDataCache() &&
             cctx.kernalContext().igfsHelper().isIgfsBlockKey(key.value(cctx.cacheObjectContext(), false))) {
             int newSize = valueLength0(val, null);
-            int oldSize = valueLength0(this.val, (this.val == null && valPtr != 0) ? valueBytes0() : null);
+            int oldSize = valueLength0(this.val, (this.val == null && hasOffHeapPointer()) ? valueBytes0() : null);
 
             int delta = newSize - oldSize;
 
@@ -199,7 +196,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (!isOffHeapValuesOnly()) {
             this.val = val;
 
-            valPtr = 0;
+            offHeapPointer(0);
         }
         else {
             try {
@@ -227,12 +224,12 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 if (val != null) {
                     byte type = val.type();
 
-                    valPtr = mem.putOffHeap(valPtr, val.valueBytes(cctx.cacheObjectContext()), type);
+                    offHeapPointer(mem.putOffHeap(offHeapPointer(), val.valueBytes(cctx.cacheObjectContext()), type));
                 }
                 else {
-                    mem.removeOffHeap(valPtr);
+                    mem.removeOffHeap(offHeapPointer());
 
-                    valPtr = 0;
+                    offHeapPointer(0);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -270,7 +267,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
         CacheObject val0 = val;
 
-        if (val0 == null && valPtr != 0) {
+        if (val0 == null && hasOffHeapPointer()) {
             IgniteBiTuple<byte[], Byte> t = valueBytes0();
 
             return cctx.cacheObjects().toCacheObject(cctx.cacheObjectContext(), t.get2(), t.get1());
@@ -434,16 +431,16 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                     if (e != null) {
                         if (e.offheapPointer() > 0) {
-                            valPtr = e.offheapPointer();
+                            offHeapPointer(e.offheapPointer());
 
                             if (needVal) {
-                                CacheObject val = cctx.fromOffheap(valPtr, false);
+                                CacheObject val = cctx.fromOffheap(offHeapPointer(), false);
 
                                 e.value(val);
                             }
                         }
                         else // Read from swap.
-                            valPtr = 0;
+                            offHeapPointer(0);
                     }
                 }
                 else
@@ -468,7 +465,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                         // Must update valPtr again since update() will reset it.
                         if (cctx.offheapTiered() && e.offheapPointer() > 0)
-                            valPtr = e.offheapPointer();
+                            offHeapPointer(e.offheapPointer());
 
                         return val;
                     }
@@ -495,13 +492,13 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 if (cctx.offheapTiered()) {
                     cctx.swap().removeOffheap(key);
 
-                    valPtr = 0;
+                    offHeapPointer(0);
                 }
 
                 return;
             }
 
-            if (val == null && cctx.offheapTiered() && valPtr != 0) {
+            if (val == null && cctx.offheapTiered() && hasOffHeapPointer()) {
                 if (log.isDebugEnabled())
                     log.debug("Value did not change, skip write swap entry: " + this);
 
@@ -540,10 +537,10 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     protected IgniteBiTuple<byte[], Byte> valueBytes0() {
         assert Thread.holdsLock(this);
 
-        if (valPtr != 0) {
+        if (hasOffHeapPointer()) {
             assert isOffHeapValuesOnly() || cctx.offheapTiered();
 
-            return cctx.unsafeMemory().get(valPtr);
+            return cctx.unsafeMemory().get(offHeapPointer());
         }
         else {
             assert val != null;
@@ -672,7 +669,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
             CacheObject val = this.val;
 
-            hasOldBytes = valPtr != 0;
+            hasOldBytes = hasOffHeapPointer();
 
             if ((unmarshal || isOffHeapValuesOnly()) && !expired && val == null && hasOldBytes)
                 val = rawGetOrUnmarshalUnlocked(tmp);
@@ -816,7 +813,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         // Update indexes before actual write to entry.
                         updateIndex(ret, expTime, nextVer, prevVal);
 
-                    boolean hadValPtr = valPtr != 0;
+                    boolean hadValPtr = hasOffHeapPointer();
 
                     // Don't change version for read-through.
                     update(ret, expTime, ttl, nextVer);
@@ -1164,7 +1161,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             // can be updated without actually holding entry lock.
             clearIndex(old);
 
-            boolean hadValPtr = valPtr != 0;
+            boolean hadValPtr = hasOffHeapPointer();
 
             update(null, 0, 0, newVer);
 
@@ -1521,7 +1518,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                     // Must persist inside synchronization in non-tx mode.
                     cctx.store().remove(null, keyValue(false));
 
-                boolean hasValPtr = valPtr != 0;
+                boolean hasValPtr = hasOffHeapPointer();
 
                 // Update index inside synchronization since it can be updated
                 // in load methods without actually holding entry lock.
@@ -2122,7 +2119,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                 enqueueVer = newVer;
 
-                boolean hasValPtr = valPtr != 0;
+                boolean hasValPtr = hasOffHeapPointer();
 
                 // Clear value on backup. Entry will be removed from cache when it got evicted from queue.
                 update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, newVer);
@@ -2929,8 +2926,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (val != null)
             return val;
 
-        if (valPtr != 0) {
-            CacheObject val0 = cctx.fromOffheap(valPtr, tmp);
+        if (hasOffHeapPointer()) {
+            CacheObject val0 = cctx.fromOffheap(offHeapPointer(), tmp);
 
             if (!tmp && cctx.kernalContext().config().isPeerClassLoadingEnabled())
                 val0.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
@@ -2952,7 +2949,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     protected boolean hasValueUnlocked() {
         assert Thread.holdsLock(this);
 
-        return val != null || valPtr != 0;
+        return val != null || hasOffHeapPointer();
     }
 
     /** {@inheritDoc} */
@@ -3298,7 +3295,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             synchronized (this) {
                 CacheObject expiredVal = saveValueForIndexUnlocked();
 
-                boolean hasOldBytes = valPtr != 0;
+                boolean hasOldBytes = hasOffHeapPointer();
 
                 boolean expired = checkExpired();
 
@@ -4102,6 +4099,27 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     }
 
     /**
+     * @return True if entry has off-heap value pointer.
+     */
+    protected boolean hasOffHeapPointer() {
+        return false;
+    }
+
+    /**
+     * @return Off-heap value pointer.
+     */
+    protected long offHeapPointer() {
+        return 0;
+    }
+
+    /**
+     * @param valPtr Off-heap value pointer.
+     */
+    protected void offHeapPointer(long valPtr) {
+        // No-op.
+    }
+
+    /**
      * @return Size of extras object.
      */
     private int extrasSize() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 1c46fd0..23060e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -102,6 +102,9 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.useOffheapEntry())
+                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });
@@ -343,17 +346,24 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     public GridCacheEntryEx entryExx(KeyCacheObject key, AffinityTopologyVersion topVer, boolean allowDetached, boolean touch) {
         try {
             return allowDetached && !ctx.affinity().localNode(key, topVer) ?
-                new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0) :
-                entryEx(key, touch);
+                createEntry(key) : entryEx(key, touch);
         }
         catch (GridDhtInvalidPartitionException e) {
             if (!allowDetached)
                 throw e;
 
-            return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0);
+            return createEntry(key);
         }
     }
 
+    /**
+     * @param key Key for which entry should be returned.
+     * @return Cache entry.
+     */
+    protected GridDistributedCacheEntry createEntry(KeyCacheObject key) {
+        return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0);
+    }
+
     /** {@inheritDoc} */
     @Override public void localLoad(Collection<? extends K> keys, final ExpiryPolicy plc)
         throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
new file mode 100644
index 0000000..1191d83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Replicated cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
+    /** Off-heap value pointer. */
+    private long valPtr;
+
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @param key    Cache key.
+     * @param hash   Key hash value.
+     * @param val    Entry value.
+     * @param next   Next entry in the linked list.
+     * @param hdrId  Header id.
+     */
+    public GridDhtOffHeapCacheEntry(GridCacheContext ctx,
+        AffinityTopologyVersion topVer,
+        KeyCacheObject key,
+        int hash,
+        CacheObject val,
+        GridCacheMapEntry next,
+        int hdrId) {
+        super(ctx, topVer, key, hash, val, next, hdrId);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasOffHeapPointer() {
+        return valPtr != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long offHeapPointer() {
+        return valPtr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void offHeapPointer(long valPtr) {
+        this.valPtr = valPtr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
index 8da4da5..098ec97 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
@@ -86,7 +86,9 @@ public class GridNoStorageCacheMap extends GridCacheConcurrentMap {
         boolean create)
     {
         if (create) {
-            GridCacheMapEntry entry = new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
+            GridCacheMapEntry entry = ctx.useOffheapEntry() ?
+                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0) :
+                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
 
             return new GridTriple<>(entry, null, null);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 905f7bf..19d88e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -124,6 +124,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.useOffheapEntry())
+                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
new file mode 100644
index 0000000..91a8e65
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * DHT atomic cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
+    /** Off-heap value pointer. */
+    private long valPtr;
+
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @param key    Cache key.
+     * @param hash   Key hash value.
+     * @param val    Entry value.
+     * @param next   Next entry in the linked list.
+     * @param hdrId  Header id.
+     */
+    public GridDhtAtomicOffHeapCacheEntry(GridCacheContext ctx,
+        AffinityTopologyVersion topVer,
+        KeyCacheObject key,
+        int hash,
+        CacheObject val,
+        GridCacheMapEntry next,
+        int hdrId) {
+        super(ctx, topVer, key, hash, val, next, hdrId);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasOffHeapPointer() {
+        return valPtr != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long offHeapPointer() {
+        return valPtr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void offHeapPointer(long valPtr) {
+        this.valPtr = valPtr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index c92d9ce..05b3c7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -86,6 +86,9 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.useOffheapEntry())
+                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });
@@ -126,7 +129,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         boolean allowDetached
     ) {
         return allowDetached && !ctx.affinity().primary(ctx.localNode(), key, topVer) ?
-            new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0) : entryExx(key, topVer);
+            createEntry(key) : entryExx(key, topVer);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
new file mode 100644
index 0000000..ed842ad
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Cache entry for colocated cache for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntry {
+    /** Off-heap value pointer. */
+    private long valPtr;
+
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @param key    Cache key.
+     * @param hash   Key hash value.
+     * @param val    Entry value.
+     * @param next   Next entry in the linked list.
+     * @param hdrId  Header id.
+     */
+    public GridDhtColocatedOffHeapCacheEntry(GridCacheContext ctx,
+        AffinityTopologyVersion topVer,
+        KeyCacheObject key,
+        int hash,
+        CacheObject val,
+        GridCacheMapEntry next,
+        int hdrId) {
+        super(ctx, topVer, key, hash, val, next, hdrId);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasOffHeapPointer() {
+        return valPtr != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long offHeapPointer() {
+        return valPtr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void offHeapPointer(long valPtr) {
+        this.valPtr = valPtr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 145e980..8258b14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -76,6 +76,9 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
             ) {
                 // Can't hold any locks here - this method is invoked when
                 // holding write-lock on the whole cache map.
+                if (ctx.useOffheapEntry())
+                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val, next, hdrId);
+
                 return new GridNearCacheEntry(ctx, key, hash, val, next, hdrId);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
new file mode 100644
index 0000000..25eb869
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Near cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
+    /** Off-heap value pointer. */
+    private long valPtr;
+
+    /**
+     * @param ctx   Cache context.
+     * @param key   Cache key.
+     * @param hash  Key hash value.
+     * @param val   Entry value.
+     * @param next  Next entry in the linked list.
+     * @param hdrId Header id.
+     */
+    public GridNearOffHeapCacheEntry(GridCacheContext ctx,
+        KeyCacheObject key,
+        int hash,
+        CacheObject val,
+        GridCacheMapEntry next,
+        int hdrId) {
+        super(ctx, key, hash, val, next, hdrId);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasOffHeapPointer() {
+        return valPtr != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long offHeapPointer() {
+        return valPtr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void offHeapPointer(long valPtr) {
+        this.valPtr = valPtr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index 84d4c90..ea59f1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -29,6 +29,9 @@ import static org.apache.ignite.events.EventType.*;
  */
 @SuppressWarnings({"NonPrivateFieldAccessedInSynchronizedContext", "TooBroadScope"})
 public class GridLocalCacheEntry extends GridCacheMapEntry {
+    /** Off-heap value pointer. */
+    private long valPtr;
+
     /**
      * @param ctx  Cache registry.
      * @param key  Cache key.
@@ -384,4 +387,19 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
 
         return doomed != null;
     }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasOffHeapPointer() {
+        return valPtr != 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long offHeapPointer() {
+        return valPtr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void offHeapPointer(long valPtr) {
+        this.valPtr = valPtr;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
new file mode 100644
index 0000000..8c7d33d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.processors.cache.local.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Cache map entry self test.
+ */
+public class CacheOffheapMapEntrySelfTest extends GridCacheAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        // No-op.
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @param memoryMode Memory mode.
+     * @param atomicityMode Atomicity mode.
+     * @param cacheMode Cache mode.
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     * @throws Exception If failed.
+     */
+    private CacheConfiguration cacheConfiguration(String gridName,
+        CacheMemoryMode memoryMode,
+        CacheAtomicityMode atomicityMode,
+        CacheMode cacheMode,
+        String cacheName)
+        throws Exception
+    {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setCacheMode(cacheMode);
+        cfg.setAtomicityMode(atomicityMode);
+        cfg.setMemoryMode(memoryMode);
+        cfg.setName(cacheName);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheMapEntry() throws Exception {
+        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, ATOMIC, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, TRANSACTIONAL, LOCAL, GridLocalCacheEntry.class);
+
+        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, PARTITIONED, GridNearCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, ATOMIC, PARTITIONED, GridNearOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, ATOMIC, PARTITIONED, GridNearOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, PARTITIONED, GridNearCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, TRANSACTIONAL, PARTITIONED, GridNearOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, TRANSACTIONAL, PARTITIONED, GridNearOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(ONHEAP_TIERED, ATOMIC, REPLICATED, GridDhtAtomicCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, ATOMIC, REPLICATED, GridDhtAtomicOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, ATOMIC, REPLICATED, GridDhtAtomicOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(ONHEAP_TIERED, TRANSACTIONAL, REPLICATED, GridDhtColocatedCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_TIERED, TRANSACTIONAL, REPLICATED, GridDhtColocatedOffHeapCacheEntry.class);
+
+        checkCacheMapEntry(OFFHEAP_VALUES, TRANSACTIONAL, REPLICATED, GridDhtColocatedOffHeapCacheEntry.class);
+    }
+
+    /**
+     * @param memoryMode Cache memory mode.
+     * @param atomicityMode Cache atomicity mode.
+     * @param cacheMode Cache mode.
+     * @param entryCls Class of cache map entry.
+     * @throws Exception If failed.
+     */
+    private void checkCacheMapEntry(CacheMemoryMode memoryMode,
+        CacheAtomicityMode atomicityMode,
+        CacheMode cacheMode,
+        Class<?> entryCls)
+        throws Exception
+    {
+        log.info("Test cache [memMode=" + memoryMode +
+            ", atomicityMode=" + atomicityMode +
+            ", cacheMode=" + cacheMode + ']');
+
+        CacheConfiguration cfg = cacheConfiguration(grid(0).name(),
+            memoryMode,
+            atomicityMode,
+            cacheMode,
+            "Cache");
+
+        try (IgniteCache jcache = grid(0).getOrCreateCache(cfg)) {
+            GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(0)).internalCache(jcache.getName());
+
+            Integer key = primaryKey(grid(0).cache(null));
+
+            cache.put(key, "val");
+
+            GridCacheEntryEx entry = cache.entryEx(key);
+
+            entry.unswap(true);
+
+            assertNotNull(entry);
+
+            assertEquals(entry.getClass(), entryCls);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d87f6d61/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index f7272d4..8eb0688 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -126,6 +126,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
 
+        suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
+
         return suite;
     }
 }



[2/8] incubator-ignite git commit: # ignite-835

Posted by vo...@apache.org.
# ignite-835


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

Branch: refs/heads/ignite-gg-9614
Commit: 9e670b8f59dc8b758dc8944885adf739d847e9c9
Parents: ade75f9
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 13 11:22:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 13 11:22:12 2015 +0300

----------------------------------------------------------------------
 ...achePartitionedNearDisabledLockSelfTest.java | 47 ++++++++++++++++++++
 .../GridCacheReplicatedLockSelfTest.java        |  5 +++
 .../testsuites/IgniteCacheTestSuite2.java       |  1 +
 3 files changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e670b8f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledLockSelfTest.java
new file mode 100644
index 0000000..69c7909
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledLockSelfTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+
+/**
+ *
+ */
+public class GridCachePartitionedNearDisabledLockSelfTest extends GridCachePartitionedLockSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = super.cacheConfiguration();
+
+        assertNotNull(ccfg.getNearConfiguration());
+
+        ccfg.setNearConfiguration(null);
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isPartitioned() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testLockReentrancy() throws Throwable {
+        fail("https://issues.apache.org/jira/browse/IGNITE-835");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e670b8f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedLockSelfTest.java
index 97df1f0..510fa0c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedLockSelfTest.java
@@ -30,4 +30,9 @@ public class GridCacheReplicatedLockSelfTest extends GridCacheLockAbstractTest {
     @Override protected CacheMode cacheMode() {
         return REPLICATED;
     }
+
+    /** {@inheritDoc} */
+    @Override public void testLockReentrancy() throws Throwable {
+        fail("https://issues.apache.org/jira/browse/IGNITE-835");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e670b8f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index dfd88a8..dc3a2c0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -76,6 +76,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCachePartitionedNearDisabledBasicStoreMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedEventSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedLockSelfTest.class));
+        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledLockSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedMultiNodeLockSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedMultiThreadedPutGetSelfTest.class));


[6/8] incubator-ignite git commit: # ignite-sprint-5 added test for IGNITE-882

Posted by vo...@apache.org.
# ignite-sprint-5 added test for IGNITE-882


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

Branch: refs/heads/ignite-gg-9614
Commit: 82467880eb18bbd16a26b92300b5212f37fe3200
Parents: 22341a9
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 13 16:16:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 13 16:16:12 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpDiscoveryRestartTest.java  | 199 +++++++++++++++++++
 1 file changed, 199 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/82467880/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
new file mode 100644
index 0000000..e6bee4a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.eclipse.jetty.util.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public class TcpDiscoveryRestartTest extends GridCommonAbstractTest {
+    /** */
+    private TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static AtomicReference<String> err;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(spi);
+
+        int[] evts = {EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT};
+
+        cfg.setIncludeEventTypes(evts);
+
+        Map<IgnitePredicate<? extends Event>, int[]> lsnrs = new HashMap<>();
+
+        lsnrs.put(new TestEventListener(), evts);
+
+        cfg.setLocalEventListeners(lsnrs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestart() throws Exception {
+        err = new AtomicReference<>();
+
+        final int NODE_CNT = 3;
+
+        startGrids(NODE_CNT);
+
+        final ConcurrentHashSet<UUID> nodeIds = new ConcurrentHashSet<>();
+
+        final AtomicInteger id = new AtomicInteger(NODE_CNT);
+
+        final IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int nodeIdx = id.getAndIncrement();
+
+                for (int i = 0; i < 10 && err.get() == null; i++) {
+                    Ignite ignite = startGrid(nodeIdx);
+
+                    UUID nodeId = ignite.cluster().localNode().id();
+
+                    if (!nodeIds.add(nodeId))
+                        failed("Duplicated node ID: " + nodeId);
+
+                    stopGrid(nodeIdx);
+                }
+
+                return null;
+            }
+        }, 5, "restart-thread");
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runMultiThreadedAsync(new Callable<Long>() {
+            @Override public Long call() throws Exception {
+                long dummyRes = 0;
+
+                List<String> list = new ArrayList<>();
+
+                while (!fut.isDone()) {
+                    for (int i = 0; i < 100; i++) {
+                        String str = new String(new byte[i]);
+
+                        list.add(str);
+
+                        dummyRes += str.hashCode();
+                    }
+
+                    if (list.size() > 1000_000) {
+                        list = new ArrayList<>();
+
+                        System.gc();
+                    }
+                }
+
+                return dummyRes;
+            }
+        }, 2, "test-load");
+
+        fut.get();
+
+        loadFut.get();
+
+        assertNull(err.get());
+
+        for (int i = 0; i < NODE_CNT; i++) {
+            Ignite ignite = ignite(i);
+
+            TestEventListener lsnr = (TestEventListener)F.firstKey(ignite.configuration().getLocalEventListeners());
+
+            assertNotNull(lsnr);
+
+            for (UUID nodeId : nodeIds)
+                lsnr.checkEvents(nodeId);
+        }
+    }
+
+
+    /**
+     * @param msg Message.
+     */
+    private void failed(String msg) {
+        info(msg);
+
+        err.compareAndSet(null, msg);
+    }
+
+    /**
+     *
+     */
+    private class TestEventListener implements IgnitePredicate<Event> {
+        /** */
+        private final ConcurrentHashSet<UUID> joinIds = new ConcurrentHashSet<>();
+
+        /** */
+        private final ConcurrentHashSet<UUID> leftIds = new ConcurrentHashSet<>();
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Event evt) {
+            DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+            if (evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT) {
+                if (!leftIds.add(evt0.eventNode().id()))
+                    failed("Duplicated failed node ID: " + evt0.eventNode().id());
+            }
+            else {
+                assertEquals(EVT_NODE_JOINED, evt.type());
+
+                if (!joinIds.add(evt0.eventNode().id()))
+                    failed("Duplicated joined node ID: " + evt0.eventNode().id());
+            }
+
+            return true;
+        }
+
+        /**
+         * @param nodeId Node ID.
+         */
+        void checkEvents(UUID nodeId) {
+            assertTrue("No join event: " + nodeId, joinIds.contains(nodeId));
+
+            assertTrue("No left event: " + nodeId, leftIds.contains(nodeId));
+        }
+    }
+}


[8/8] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-gg-9614

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-gg-9614


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

Branch: refs/heads/ignite-gg-9614
Commit: 98008911ea938e795b8be2bde6cba63101a3a080
Parents: 436227d d87f6d6
Author: ptupitsyn <pt...@gridgain.com>
Authored: Wed May 13 17:47:04 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Wed May 13 17:47:04 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   7 +
 .../processors/cache/GridCacheMapEntry.java     |  70 ++++---
 .../distributed/dht/GridDhtCacheAdapter.java    |  16 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  63 ++++++
 .../dht/GridDhtTransactionalCacheAdapter.java   |   9 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   3 +
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  63 ++++++
 .../dht/colocated/GridDhtColocatedCache.java    |   5 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  63 ++++++
 .../distributed/near/GridNearCacheAdapter.java  |   3 +
 .../near/GridNearOffHeapCacheEntry.java         |  60 ++++++
 .../cache/local/GridLocalCacheEntry.java        |  18 ++
 .../cache/CacheOffheapMapEntrySelfTest.java     | 168 ++++++++++++++++
 .../distributed/GridCacheLockAbstractTest.java  |   2 -
 ...achePartitionedNearDisabledLockSelfTest.java |  47 +++++
 .../GridCachePartitionedTxSalvageSelfTest.java  |  25 +--
 .../GridCacheReplicatedLockSelfTest.java        |   5 +
 .../discovery/tcp/TcpDiscoveryRestartTest.java  | 199 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 21 files changed, 783 insertions(+), 50 deletions(-)
----------------------------------------------------------------------



[3/8] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-835

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-835


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

Branch: refs/heads/ignite-gg-9614
Commit: 96ff8f2fc6521f55300ab634584f8f25c6edfdf2
Parents: 9e670b8 bbc21a6
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 13 11:23:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 13 11:23:20 2015 +0300

----------------------------------------------------------------------
 bin/include/functions.sh                        |    2 +-
 .../internal/direct/DirectByteBufferStream.java |    4 +-
 .../eventstorage/GridEventStorageManager.java   |    5 +-
 .../processors/cache/GridCacheMvccManager.java  |    4 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   32 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   27 +
 .../cache/distributed/dht/GridDhtTxMapping.java |    2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   81 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   25 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |    4 +-
 .../distributed/near/GridNearCacheEntry.java    |    4 +-
 .../distributed/near/GridNearLockFuture.java    |    5 -
 .../near/GridNearOptimisticTxPrepareFuture.java |  779 +++++++++++++
 .../GridNearPessimisticTxPrepareFuture.java     |  349 ++++++
 .../cache/distributed/near/GridNearTxLocal.java |   84 +-
 .../near/GridNearTxPrepareFuture.java           | 1050 ------------------
 .../near/GridNearTxPrepareFutureAdapter.java    |  226 ++++
 .../cache/transactions/IgniteInternalTx.java    |    4 +-
 .../cache/transactions/IgniteTxAdapter.java     |    2 +-
 .../cache/transactions/IgniteTxHandler.java     |   68 +-
 .../transactions/IgniteTxLocalAdapter.java      |    2 +-
 .../cache/transactions/IgniteTxManager.java     |   12 +-
 .../ignite/internal/util/IgniteUtils.java       |    4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    8 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |   11 +-
 .../distributed/IgniteTxGetAfterStopTest.java   |  131 +++
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |    4 +-
 ...idCacheAtomicReplicatedFailoverSelfTest.java |    6 +
 .../GridCacheReplicatedFailoverSelfTest.java    |    6 +
 .../GridCacheReplicatedNodeRestartSelfTest.java |   80 ++
 .../IgniteCacheFailoverTestSuite.java           |   10 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |    8 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    2 +
 33 files changed, 1765 insertions(+), 1276 deletions(-)
----------------------------------------------------------------------



[4/8] incubator-ignite git commit: # ignite-sprint-5 minor test change

Posted by vo...@apache.org.
# ignite-sprint-5 minor test change


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

Branch: refs/heads/ignite-gg-9614
Commit: 5f8925be6d6e1360ff56ff3c1dfff49811caef82
Parents: bbc21a6
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 13 12:38:49 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 13 12:38:49 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedTxSalvageSelfTest.java  | 25 ++++++++++----------
 1 file changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8925be/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 5072642..6192f39 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -35,6 +34,8 @@ import org.apache.ignite.transactions.*;
 import java.util.*;
 
 import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
 import static org.apache.ignite.transactions.TransactionConcurrency.*;
 import static org.apache.ignite.transactions.TransactionIsolation.*;
 
@@ -76,10 +77,10 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
 
         CacheConfiguration cc = defaultCacheConfiguration();
 
-        cc.setCacheMode(CacheMode.PARTITIONED);
+        cc.setCacheMode(PARTITIONED);
         cc.setAffinity(new RendezvousAffinityFunction(false, 18));
         cc.setBackups(1);
-        cc.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cc.setRebalanceMode(SYNC);
 
         c.setCacheConfiguration(cc);
 
@@ -109,8 +110,9 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        // Shutwodn the gird.
         stopAllGrids();
+
+        System.gc();
     }
 
     /**
@@ -145,8 +147,8 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      * Check whether caches has no transactions after salvage timeout.
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
-     * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
+     * @param prepare Whether to prepare transaction state
+     *                (i.e. call {@link IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void checkSalvageAfterTimeout(TransactionConcurrency mode, boolean prepare) throws Exception {
@@ -164,8 +166,8 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      * Check whether caches still has all transactions before salvage timeout.
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
-     * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
+     * @param prepare Whether to prepare transaction state
+     *                (i.e. call {@link IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void checkSalvageBeforeTimeout(TransactionConcurrency mode, boolean prepare) throws Exception {
@@ -191,8 +193,8 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
      * Start new transaction on the grid(0) and put some keys to it.
      *
      * @param mode Transaction mode (PESSIMISTIC, OPTIMISTIC).
-     * @param prepare Whether to preapre transaction state
-     *                (i.e. call {@link org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx#prepare()}).
+     * @param prepare Whether to prepare transaction state
+     *                (i.e. call {@link IgniteInternalTx#prepare()}).
      * @throws Exception If failed.
      */
     private void startTxAndPutKeys(final TransactionConcurrency mode, final boolean prepare) throws Exception {
@@ -210,9 +212,8 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
                     for (Integer key : keys)
                         c.put(key, "val" + key);
 
-                    // Unproxy.
                     if (prepare)
-                        U.<IgniteInternalTx>field(tx, "tx").prepare();
+                        ((TransactionProxyImpl)tx).tx().prepare();
                 }
                 catch (IgniteCheckedException e) {
                     info("Failed to put keys to cache: " + e.getMessage());