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/09/10 14:04:36 UTC

[01/17] ignite git commit: ignite-971 Fix offheap to swap eviction.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1282 87fd9acef -> 836283df0


ignite-971 Fix offheap to swap eviction.


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

Branch: refs/heads/ignite-1282
Commit: b92221643729be85863d180b14e83c7268dafae4
Parents: 60a76bc
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 9 16:32:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 9 17:45:37 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheEntryEx.java      |  13 ++
 .../cache/GridCacheEvictionManager.java         |  18 +-
 .../processors/cache/GridCacheMapEntry.java     |  45 ++++-
 .../cache/GridCacheSwapEntryImpl.java           |  24 +--
 .../processors/cache/GridCacheSwapManager.java  | 175 +++++++++++++++----
 .../processors/cache/GridCacheUtils.java        |  17 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +-
 .../offheap/GridOffHeapProcessor.java           |  25 ++-
 .../util/offheap/GridOffHeapEvictListener.java  |   5 +
 .../internal/util/offheap/GridOffHeapMap.java   |  13 +-
 .../util/offheap/GridOffHeapMapFactory.java     |  28 +--
 .../util/offheap/GridOffHeapPartitionedMap.java |  11 ++
 .../util/offheap/unsafe/GridUnsafeMap.java      | 127 +++++++++-----
 .../unsafe/GridUnsafePartitionedMap.java        |   9 +
 .../cache/CacheSwapUnswapGetTest.java           |  85 ++++++++-
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 .../offheap/GridOffHeapMapAbstractSelfTest.java |  16 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |  20 +++
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../offheap/unsafe/GridUnsafeMapSelfTest.java   |   2 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   9 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 -
 25 files changed, 521 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 98e86ed..430590a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -867,6 +867,19 @@ public interface GridCacheEntryEx {
     public void updateTtl(@Nullable GridCacheVersion ver, long ttl);
 
     /**
+     * Tries to do offheap -> swap eviction.
+     *
+     * @param entry Serialized swap entry.
+     * @param evictVer Version when entry was selected for eviction.
+     * @param obsoleteVer Obsolete version.
+     * @throws IgniteCheckedException If failed.
+     * @throws GridCacheEntryRemovedException If entry was removed.
+     * @return {@code True} if entry was obsoleted and written to swap.
+     */
+    public boolean offheapSwapEvict(byte[] entry, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
+
+    /**
      * @return Value.
      * @throws IgniteCheckedException If failed to read from swap storage.
      * @throws GridCacheEntryRemovedException If entry was removed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index f60c0eb..3e0e2f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -958,7 +958,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
         List<GridCacheEntryEx> locked = new ArrayList<>(keys.size());
 
-        Set<GridCacheEntryEx> notRemove = null;
+        Set<GridCacheEntryEx> notRmv = null;
 
         Collection<GridCacheBatchSwapEntry> swapped = new ArrayList<>(keys.size());
 
@@ -990,10 +990,10 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                 locked.add(entry);
 
                 if (entry.obsolete()) {
-                    if (notRemove == null)
-                        notRemove = new HashSet<>();
+                    if (notRmv == null)
+                        notRmv = new HashSet<>();
 
-                    notRemove.add(entry);
+                    notRmv.add(entry);
 
                     continue;
                 }
@@ -1004,11 +1004,19 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                 GridCacheBatchSwapEntry swapEntry = entry.evictInBatchInternal(obsoleteVer);
 
                 if (swapEntry != null) {
+                    assert entry.obsolete() : entry;
+
                     swapped.add(swapEntry);
 
                     if (log.isDebugEnabled())
                         log.debug("Entry was evicted [entry=" + entry + ", localNode=" + cctx.nodeId() + ']');
                 }
+                else if (!entry.obsolete()) {
+                    if (notRmv == null)
+                        notRmv = new HashSet<>();
+
+                    notRmv.add(entry);
+                }
             }
 
             // Batch write to swap.
@@ -1025,7 +1033,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
 
             // Remove entries and fire events outside the locks.
             for (GridCacheEntryEx entry : locked) {
-                if (entry.obsolete() && (notRemove == null || !notRemove.contains(entry))) {
+                if (entry.obsolete() && (notRmv == null || !notRmv.contains(entry))) {
                     entry.onMarkedObsolete();
 
                     cache.removeEntry(entry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/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 eb4d864..3fc1b2e 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
@@ -433,6 +433,43 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
+    @Override public boolean offheapSwapEvict(byte[] vb, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        assert cctx.swap().swapEnabled() && cctx.swap().offHeapEnabled() : this;
+
+        boolean obsolete;
+
+        synchronized (this) {
+            checkObsolete();
+
+            if (hasReaders() || !isStartVersion())
+                return false;
+
+            GridCacheMvcc mvcc = mvccExtras();
+
+            if (mvcc != null && !mvcc.isEmpty(obsoleteVer))
+                return false;
+
+            if (cctx.swap().removeOffheap(key, partition(), evictVer)) {
+                assert !hasValueUnlocked() : this;
+
+                obsolete = markObsolete0(obsoleteVer, false);
+
+                assert obsolete : this;
+
+                cctx.swap().writeToSwap(partition(), key, vb);
+            }
+            else
+                obsolete = false;
+        }
+
+        if (obsolete)
+            onMarkedObsolete();
+
+        return obsolete;
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject unswap() throws IgniteCheckedException, GridCacheEntryRemovedException {
         return unswap(true);
     }
@@ -536,7 +573,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     log.debug("Value did not change, skip write swap entry: " + this);
 
                 if (cctx.swap().offheapEvictionEnabled())
-                    cctx.swap().enableOffheapEviction(key());
+                    cctx.swap().enableOffheapEviction(key(), partition());
 
                 return;
             }
@@ -3643,6 +3680,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         try {
             if (F.isEmptyOrNulls(filter)) {
                 synchronized (this) {
+                    if (obsoleteVersionExtras() != null)
+                        return false;
+
                     CacheObject prev = saveValueForIndexUnlocked();
 
                     if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
@@ -3684,6 +3724,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         return false;
 
                     synchronized (this) {
+                        if (obsoleteVersionExtras() != null)
+                            return false;
+
                         if (!v.equals(ver))
                             // Version has changed since entry passed the filter. Do it again.
                             continue;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
index 81490a7..b7c66d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
@@ -126,9 +126,9 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return Version.
      */
     public static GridCacheVersion version(byte[] bytes) {
-        int off = VERSION_OFFSET; // Skip ttl, expire time.
+        long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
 
-        boolean verEx = bytes[off++] != 0;
+        boolean verEx = UNSAFE.getByte(bytes, off++) != 0;
 
         return U.readVersion(bytes, off, verEx);
     }
@@ -157,26 +157,6 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
         return new IgniteBiTuple<>(valBytes, type);
     }
 
-    /**
-     * @param bytes Entry bytes.
-     * @return Value bytes offset.
-     */
-    public static int valueOffset(byte[] bytes) {
-        assert bytes.length > 40 : bytes.length;
-
-        int off = VERSION_OFFSET; // Skip ttl, expire time.
-
-        boolean verEx = bytes[off++] != 0;
-
-        off += verEx ? VERSION_EX_SIZE : VERSION_SIZE;
-
-        off += 5; // Byte array flag + array size.
-
-        assert bytes.length >= off;
-
-        return off;
-    }
-
     /** {@inheritDoc} */
     @Override public byte[] valueBytes() {
         if (valBytes != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 7fd6013..53b0421 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -54,6 +54,7 @@ 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.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.swapspace.SwapKey;
@@ -101,8 +102,13 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     private final ReferenceQueue<Iterator<Map.Entry>> itQ = new ReferenceQueue<>();
 
     /** Soft iterator set. */
-    private final Collection<GridWeakIterator<Map.Entry>> itSet =
-        new GridConcurrentHashSet<>();
+    private final Collection<GridWeakIterator<Map.Entry>> itSet = new GridConcurrentHashSet<>();
+
+    /** {@code True} if offheap to swap eviction is possible. */
+    private boolean offheapToSwapEvicts;
+
+    /** Values to be evicted from offheap to swap. */
+    private ThreadLocal<Collection<IgniteBiTuple<byte[], byte[]>>> offheapEvicts = new ThreadLocal<>();
 
     /**
      * @param enabled Flag to indicate if swap is enabled.
@@ -127,9 +133,58 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     *
+     */
+    public void unwindOffheapEvicts() {
+        if (!offheapToSwapEvicts)
+            return;
+
+        Collection<IgniteBiTuple<byte[], byte[]>> evicts = offheapEvicts.get();
+
+        if (evicts != null) {
+            GridCacheVersion obsoleteVer = cctx.versions().next();
+
+            for (IgniteBiTuple<byte[], byte[]> t : evicts) {
+                try {
+                    byte[] kb = t.get1();
+                    byte[] vb = t.get2();
+
+                    GridCacheVersion evictVer = GridCacheSwapEntryImpl.version(vb);
+
+                    KeyCacheObject key = cctx.toCacheKeyObject(kb);
+
+                    while (true) {
+                        GridCacheEntryEx entry = cctx.cache().entryEx(key);
+
+                        try {
+                            if (entry.offheapSwapEvict(vb, evictVer, obsoleteVer))
+                                cctx.cache().removeEntry(entry);
+
+                            break;
+                        }
+                        catch (GridCacheEntryRemovedException ignore) {
+                            // Retry.
+                        }
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to unmarshal off-heap entry", e);
+                }
+            }
+
+            offheapEvicts.set(null);
+        }
+    }
+
+    /** First offheap eviction warning flag. */
+    private volatile boolean firstEvictWarn;
+
+    /**
      * Initializes off-heap space.
      */
     private void initOffHeap() {
+        assert offheapEnabled;
+
         // Register big data usage.
         long max = cctx.config().getOffHeapMaxMemory();
 
@@ -137,43 +192,69 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int parts = cctx.config().getAffinity().partitions();
 
-        GridOffHeapEvictListener lsnr = !swapEnabled && !offheapEnabled ? null : new GridOffHeapEvictListener() {
-            private volatile boolean firstEvictWarn;
+        GridOffHeapEvictListener lsnr;
 
-            @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
-                try {
-                    if (!firstEvictWarn)
-                        warnFirstEvict();
+        if (swapEnabled) {
+            offheapToSwapEvicts = true;
 
-                    writeToSwap(part, cctx.toCacheKeyObject(kb), vb);
+            lsnr = new GridOffHeapEvictListener() {
+                @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
+                    assert offheapToSwapEvicts;
 
-                    if (cctx.config().isStatisticsEnabled())
-                        cctx.cache().metrics0().onOffHeapEvict();
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
-                }
-            }
+                    onOffheapEvict();
+
+                    Collection<IgniteBiTuple<byte[], byte[]>> evicts = offheapEvicts.get();
 
-            private void warnFirstEvict() {
-                synchronized (this) {
-                    if (firstEvictWarn)
-                        return;
+                    if (evicts == null)
+                        offheapEvicts.set(evicts = new ArrayList<>());
 
-                    firstEvictWarn = true;
+                    evicts.add(new IgniteBiTuple<>(kb, vb));
                 }
 
-                U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " +
-                    "cache configuration [cache=" + cctx.name() +
-                    ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']',
-                    "Off-heap evictions started: " + cctx.name());
-            }
-        };
+                @Override public boolean removedEvicted() {
+                    return false;
+                }
+            };
+        }
+        else {
+            lsnr = new GridOffHeapEvictListener() {
+                @Override public void onEvict(int part, int hash, byte[] kb, byte[] vb) {
+                    onOffheapEvict();
+                }
+
+                @Override public boolean removedEvicted() {
+                    return true;
+                }
+            };
+        }
 
         offheap.create(spaceName, parts, init, max, lsnr);
     }
 
     /**
+     * Warns on first evict from off-heap.
+     */
+    private void onOffheapEvict() {
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapEvict();
+
+        if (!firstEvictWarn)
+            return;
+
+        synchronized (this) {
+            if (firstEvictWarn)
+                return;
+
+            firstEvictWarn = true;
+        }
+
+        U.warn(log, "Off-heap evictions started. You may wish to increase 'offHeapMaxMemory' in " +
+            "cache configuration [cache=" + cctx.name() +
+            ", offHeapMaxMemory=" + cctx.config().getOffHeapMaxMemory() + ']',
+            "Off-heap evictions started: " + cctx.name());
+    }
+
+    /**
      * @return {@code True} if swap store is enabled.
      */
     public boolean swapEnabled() {
@@ -966,6 +1047,35 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param key Key to remove.
+     * @param part Partition.
+     * @param ver Expected version.
+     * @return {@code True} if removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    boolean removeOffheap(final KeyCacheObject key, int part, final GridCacheVersion ver)
+        throws IgniteCheckedException {
+        assert offheapEnabled;
+
+        checkIteratorQueue();
+
+        boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()),
+            new IgniteBiPredicate<Long, Integer>() {
+                @Override public boolean apply(Long ptr, Integer len) {
+                    GridCacheVersion ver0 = GridCacheOffheapSwapEntry.version(ptr);
+
+                    return ver.equals(ver0);
+                }
+            }
+        );
+
+        if (rmv && cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRemove();
+
+        return rmv;
+    }
+
+    /**
      * @return {@code True} if offheap eviction is enabled.
      */
     boolean offheapEvictionEnabled() {
@@ -976,16 +1086,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * Enables eviction for offheap entry after {@link #readOffheapPointer} was called.
      *
      * @param key Key.
+     * @param part Partition.
      * @throws IgniteCheckedException If failed.
      */
-    void enableOffheapEviction(final KeyCacheObject key) throws IgniteCheckedException {
+    void enableOffheapEviction(final KeyCacheObject key, int part) throws IgniteCheckedException {
         if (!offheapEnabled)
             return;
 
         checkIteratorQueue();
 
-        int part = cctx.affinity().partition(key);
-
         offheap.enableEviction(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
     }
 
@@ -1224,7 +1333,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @param entry Entry bytes.
      * @throws IgniteCheckedException If failed.
      */
-    private void writeToSwap(int part, KeyCacheObject key, byte[] entry) throws IgniteCheckedException {
+    public void writeToSwap(int part, KeyCacheObject key, byte[] entry) throws IgniteCheckedException {
+        assert swapEnabled;
+
         checkIteratorQueue();
 
         swapMgr.write(spaceName,
@@ -1244,7 +1355,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      */
     public void clearOffHeap() {
         if (offheapEnabled)
-            initOffHeap();
+            clearOffHeap();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 980971c..919bce6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1029,9 +1029,14 @@ public class GridCacheUtils {
 
         ctx.evicts().unwind();
 
-        if (ctx.isNear())
+        ctx.swap().unwindOffheapEvicts();
+
+        if (ctx.isNear()) {
             ctx.near().dht().context().evicts().unwind();
 
+            ctx.swap().unwindOffheapEvicts();
+        }
+
         ctx.ttl().expire();
     }
 
@@ -1041,14 +1046,8 @@ public class GridCacheUtils {
     public static <K, V> void unwindEvicts(GridCacheSharedContext<K, V> ctx) {
         assert ctx != null;
 
-        for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts()) {
-            cacheCtx.evicts().unwind();
-
-            if (cacheCtx.isNear())
-                cacheCtx.near().dht().context().evicts().unwind();
-
-            cacheCtx.ttl().expire();
-        }
+        for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts())
+            unwindEvicts(cacheCtx);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 215a1b5..3c74055 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -261,8 +261,11 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         map.put(entry.key(), entry);
 
-        if (!entry.isInternal())
+        if (!entry.isInternal()) {
+            assert !entry.deleted() : entry;
+
             mapPubSize.increment();
+        }
     }
 
     /**
@@ -270,7 +273,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
     void onRemoved(GridDhtCacheEntry entry) {
-        assert entry.obsolete();
+        assert entry.obsolete() : entry;
 
         // Make sure to remove exactly this entry.
         synchronized (entry) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
index 024ea7c..492fa07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.Marshaller;
 import org.jetbrains.annotations.Nullable;
@@ -261,13 +262,35 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
+    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
+        throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes));
     }
 
     /**
+     * Removes value from offheap space for the given key.
+     *
+     * @param spaceName Space name.
+     * @param part Partition.
+     * @param key Key.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code true} If succeeded.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean removex(@Nullable String spaceName,
+        int part,
+        KeyCacheObject key,
+        byte[] keyBytes,
+        IgniteBiPredicate<Long, Integer> p) throws IgniteCheckedException {
+        GridOffHeapPartitionedMap m = offheap(spaceName);
+
+        return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes), p);
+    }
+
+    /**
      * Gets iterator over contents of the given space.
      *
      * @param spaceName Space name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
index 4597be8..1219ae1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapEvictListener.java
@@ -30,4 +30,9 @@ public interface GridOffHeapEvictListener {
      * @param valBytes Value bytes.
      */
     public void onEvict(int part, int hash, byte[] keyBytes, byte[] valBytes);
+
+    /**
+     * @return {@code True} if entry selected for eviction should be immediately removed.
+     */
+    public boolean removedEvicted();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
index 1fcddd7..d14a582 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMap.java
@@ -20,13 +20,14 @@ package org.apache.ignite.internal.util.offheap;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Off-heap map.
  */
-public interface GridOffHeapMap<K> {
+public interface GridOffHeapMap {
     /**
      * Gets partition this map belongs to.
      *
@@ -102,6 +103,16 @@ public interface GridOffHeapMap<K> {
     public boolean removex(int hash, byte[] keyBytes);
 
     /**
+     * Removes value from off-heap map without returning it.
+     *
+     * @param hash Hash.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code True} if value was removed.
+     */
+    public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p);
+
+    /**
      * Puts key and value bytes into the map potentially replacing
      * existing entry.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
index 1a3d219..4dd911f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapFactory.java
@@ -32,8 +32,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(long initCap) {
+        return new GridUnsafeMap(128, 0.75f, initCap, 0, (short)0, null);
     }
 
     /**
@@ -43,8 +43,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, long initCap) {
-        return new GridUnsafeMap<>(concurrency, 0.75f, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(int concurrency, long initCap) {
+        return new GridUnsafeMap(concurrency, 0.75f, initCap, 0, (short)0, null);
     }
 
     /**
@@ -55,8 +55,8 @@ public class GridOffHeapMapFactory {
      * @param initCap Initial capacity.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, float load, long initCap) {
-        return new GridUnsafeMap<>(concurrency, load, initCap, 0, (short)0, null);
+    public static GridOffHeapMap unsafeMap(int concurrency, float load, long initCap) {
+        return new GridUnsafeMap(concurrency, load, initCap, 0, (short)0, null);
     }
 
     /**
@@ -68,8 +68,8 @@ public class GridOffHeapMapFactory {
      * @param lruStripes Number of LRU stripes.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap, long totalMem, short lruStripes) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, null);
+    public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes) {
+        return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, null);
     }
 
     /**
@@ -82,9 +82,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(long initCap, long totalMem, short lruStripes,
+    public static GridOffHeapMap unsafeMap(long initCap, long totalMem, short lruStripes,
         @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(128, 0.75f, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(128, 0.75f, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**
@@ -98,9 +98,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes,
+    public static GridOffHeapMap unsafeMap(int concurrency, long initCap, long totalMem, short lruStripes,
         @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(concurrency, 0.75f, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**
@@ -115,9 +115,9 @@ public class GridOffHeapMapFactory {
      * @param lsnr Optional eviction listener which gets notified every time an entry is evicted.
      * @return Off-heap map.
      */
-    public static <K> GridOffHeapMap<K> unsafeMap(int concurrency, float load, long initCap, long totalMem,
+    public static <K> GridOffHeapMap unsafeMap(int concurrency, float load, long initCap, long totalMem,
         short lruStripes, @Nullable GridOffHeapEvictListener lsnr) {
-        return new GridUnsafeMap<>(concurrency, load, initCap, totalMem, lruStripes, lsnr);
+        return new GridUnsafeMap(concurrency, load, initCap, totalMem, lruStripes, lsnr);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
index 3afdfa9..5e03677 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMap.java
@@ -21,6 +21,7 @@ import java.util.Set;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
@@ -111,6 +112,16 @@ public interface GridOffHeapPartitionedMap {
     public boolean removex(int p, int hash, byte[] keyBytes);
 
     /**
+     * Removes value from off-heap map without returning it.
+     * @param part Partition.
+     * @param hash Hash.
+     * @param keyBytes Key bytes.
+     * @param p Value predicate (arguments are value address and value length).
+     * @return {@code True} if value was removed.
+     */
+    public boolean removex(int part, int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p);
+
+    /**
      * Puts key and value bytes into the map potentially replacing
      * existing entry.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
index 40fb3e8..ed13fe1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.LongAdder8;
@@ -42,7 +43,7 @@ import static org.apache.ignite.internal.util.offheap.GridOffHeapEvent.REHASH;
 /**
  * Off-heap map based on {@code Unsafe} implementation.
  */
-public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
+public class GridUnsafeMap implements GridOffHeapMap {
     /** Header size. */
     private static final int HEADER = 4 /*hash*/ + 4 /*key-size*/  + 4 /*value-size*/ + 8 /*queue-address*/ +
         8 /*next-address*/;
@@ -77,7 +78,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     private final float load;
 
     /** Segments. */
-    private final Segment<K>[] segs;
+    private final Segment[] segs;
 
     /** Total memory. */
     private final GridUnsafeMemory mem;
@@ -111,6 +112,9 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     /** LRU poller. */
     private final GridUnsafeLruPoller lruPoller;
 
+    /** */
+    private final boolean rmvEvicted;
+
     /**
      * @param concurrency Concurrency.
      * @param load Load factor.
@@ -180,6 +184,8 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                 }
             }
         };
+
+        rmvEvicted = evictLsnr == null || evictLsnr.removedEvicted();
     }
 
     /**
@@ -225,6 +231,8 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
         segs = new Segment[size];
 
         init(initCap, size);
+
+        rmvEvicted = evictLsnr == null || evictLsnr.removedEvicted();
     }
 
     /**
@@ -247,7 +255,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
 
         for (int i = 0; i < size; i++) {
             try {
-                segs[i] = new Segment<>(i, cap);
+                segs[i] = new Segment(i, cap);
             }
             catch (GridOffHeapOutOfMemoryException e) {
                 destruct();
@@ -327,6 +335,11 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p) {
+        return segmentFor(hash).removex(hash, keyBytes, p);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean put(int hash, byte[] keyBytes, byte[] valBytes) {
         return segmentFor(hash).put(hash, keyBytes, valBytes);
     }
@@ -559,7 +572,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
     /**
      * Segment.
      */
-    private class Segment<K> {
+    private class Segment {
         /** Lock. */
         private final ReadWriteLock lock = new ReentrantReadWriteLock();
 
@@ -1009,41 +1022,44 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                         }
 
                         if (cur != 0) {
-                            long next = Entry.nextAddress(cur, mem);
-
-                            if (prev != 0)
-                                Entry.nextAddress(prev, next, mem); // Relink.
-                            else {
-                                if (next == 0)
-                                    Bin.clear(binAddr, mem);
-                                else
-                                    Bin.first(binAddr, next, mem);
-                            }
+                            long a;
 
-                            if (evictLsnr != null) {
-                                keyBytes = Entry.readKeyBytes(cur, mem);
+                            assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
+                                "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
 
-                                // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released.
-//                                valBytes = Entry.readValueBytes(cur, mem);
-                                {
-                                    int keyLen = Entry.readKeyLength(cur, mem);
-                                    int valLen = Entry.readValueLength(cur, mem);
+                            if (rmvEvicted) {
+                                long next = Entry.nextAddress(cur, mem);
 
-                                    valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                                if (prev != 0)
+                                    Entry.nextAddress(prev, next, mem); // Relink.
+                                else {
+                                    if (next == 0)
+                                        Bin.clear(binAddr, mem);
+                                    else
+                                        Bin.first(binAddr, next, mem);
                                 }
-                            }
 
-                            long a;
+                                relSize = Entry.size(cur, mem);
+                                relAddr = cur;
 
-                            assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
-                                "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
+                                cnt--;
 
-                            relSize = Entry.size(cur, mem);
-                            relAddr = cur;
+                                totalCnt.decrement();
+                            }
+                            else {
+                                if (qAddr != 0) {
+                                    boolean clear = Entry.clearQueueAddress(cur, qAddr, mem);
 
-                            cnt--;
+                                    assert clear;
+                                }
 
-                            totalCnt.decrement();
+                                keyBytes = Entry.readKeyBytes(cur, mem);
+
+                                int keyLen = Entry.readKeyLength(cur, mem);
+                                int valLen = Entry.readValueLength(cur, mem);
+
+                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                            }
                         }
                     }
                 }
@@ -1251,7 +1267,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          */
         @SuppressWarnings("TooBroadScope")
         byte[] remove(int hash, byte[] keyBytes) {
-            return remove(hash, keyBytes, true);
+            return remove(hash, keyBytes, true, null);
         }
 
         /**
@@ -1260,17 +1276,28 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          * @return {@code True} if value was removed.
          */
         boolean removex(int hash, byte[] keyBytes) {
-            return remove(hash, keyBytes, false) == EMPTY_BYTES;
+            return remove(hash, keyBytes, false, null) == EMPTY_BYTES;
+        }
+
+        /**
+         * @param hash Hash.
+         * @param keyBytes Key bytes.
+         * @param p Value predicate.
+         * @return {@code True} if value was removed.
+         */
+        boolean removex(int hash, byte[] keyBytes, IgniteBiPredicate<Long, Integer> p) {
+            return remove(hash, keyBytes, false, p) == EMPTY_BYTES;
         }
 
         /**
          * @param hash Hash.
          * @param keyBytes Key bytes.
          * @param retval {@code True} if need removed value.
+         * @param p Value predicate.
          * @return Removed value bytes.
          */
         @SuppressWarnings("TooBroadScope")
-        byte[] remove(int hash, byte[] keyBytes, boolean retval) {
+        byte[] remove(int hash, byte[] keyBytes, boolean retval, @Nullable IgniteBiPredicate<Long, Integer> p) {
             int relSize = 0;
             long relAddr = 0;
             long qAddr = 0;
@@ -1291,6 +1318,19 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
 
                         // If found match.
                         if (Entry.keyEquals(cur, keyBytes, mem)) {
+                            int keyLen = 0;
+                            int valLen = 0;
+
+                            if (p != null) {
+                                keyLen = Entry.readKeyLength(cur, mem);
+                                valLen = Entry.readValueLength(cur, mem);
+
+                                long valPtr = cur + HEADER + keyLen;
+
+                                if (!p.apply(valPtr, valLen))
+                                    return null;
+                            }
+
                             if (prev != 0)
                                 Entry.nextAddress(prev, next, mem); // Relink.
                             else {
@@ -1300,18 +1340,16 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
                                     Bin.first(binAddr, next, mem);
                             }
 
-                            // TODO: GG-8123: Inlined as a workaround. Revert when 7u60 is released.
-//                            valBytes = retval ? Entry.readValueBytes(cur, mem) : EMPTY_BYTES;
-                            {
-                                if (retval) {
-                                    int keyLen = Entry.readKeyLength(cur, mem);
-                                    int valLen = Entry.readValueLength(cur, mem);
-
-                                    valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                            if (retval) {
+                                if (keyLen == 0) {
+                                    keyLen = Entry.readKeyLength(cur, mem);
+                                    valLen = Entry.readValueLength(cur, mem);
                                 }
-                                else
-                                    valBytes = EMPTY_BYTES;
+
+                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
                             }
+                            else
+                                valBytes = EMPTY_BYTES;
 
                             // Prepare release of memory.
                             qAddr = Entry.queueAddress(cur, mem);
@@ -1382,8 +1420,7 @@ public class GridUnsafeMap<K> implements GridOffHeapMap<K> {
          * @param keyBytes Key bytes.
          * @return Value pointer.
          */
-        @Nullable
-        IgniteBiTuple<Long, Integer> valuePointer(int hash, byte[] keyBytes) {
+        @Nullable IgniteBiTuple<Long, Integer> valuePointer(int hash, byte[] keyBytes) {
             long binAddr = readLock(hash);
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
index 070da51..fb8ac14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafePartitionedMap.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMap;
 import org.apache.ignite.internal.util.offheap.GridOffHeapPartitionedMap;
 import org.apache.ignite.internal.util.typedef.CX2;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.LongAdder8;
@@ -198,6 +199,14 @@ public class GridUnsafePartitionedMap implements GridOffHeapPartitionedMap {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean removex(int part,
+        int hash,
+        byte[] keyBytes,
+        IgniteBiPredicate<Long, Integer> p) {
+        return mapFor(part).removex(hash, keyBytes, p);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean put(int p, int hash, byte[] keyBytes, byte[] valBytes) {
         return mapFor(p).put(hash, keyBytes, valBytes);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
index 271d8b1..214beb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -53,6 +54,12 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     /** */
     private static final long DURATION = 30_000;
 
+    /** */
+    private static final long OFFHEAP_MEM = 1000;
+
+    /** */
+    private static final int MAX_HEAP_SIZE = 100;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -81,7 +88,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
 
         if (memMode == CacheMemoryMode.ONHEAP_TIERED) {
             LruEvictionPolicy plc = new LruEvictionPolicy();
-            plc.setMaxSize(100);
+            plc.setMaxSize(MAX_HEAP_SIZE);
 
             ccfg.setEvictionPolicy(plc);
         }
@@ -89,7 +96,7 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
         if (swap) {
             ccfg.setSwapEnabled(true);
 
-            ccfg.setOffHeapMaxMemory(1000);
+            ccfg.setOffHeapMaxMemory(OFFHEAP_MEM);
         }
         else
             ccfg.setOffHeapMaxMemory(0);
@@ -133,6 +140,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testTxCacheOffheapSwapEvict() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCacheOffheapTieredSwapEvict() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicCacheOffheapEvict() throws Exception {
         swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, false);
     }
@@ -145,6 +166,20 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapSwapEvict() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapTieredSwapEvict() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, true);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @param memMode Cache memory mode.
      * @param swap {@code True} if swap enabled.
@@ -220,12 +255,56 @@ public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
                 }
             });
 
-            Thread.sleep(DURATION);
+            long endTime = System.currentTimeMillis() + DURATION;
+
+            while (System.currentTimeMillis() < endTime) {
+                Thread.sleep(5000);
+
+                log.info("Cache size [heap=" + cache.localSize(CachePeekMode.ONHEAP) +
+                    ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) +
+                    ", swap=" + cache.localSize(CachePeekMode.SWAP) +
+                    ", total=" + cache.localSize() +
+                    ", offheapMem=" + cache.metrics().getOffHeapAllocatedSize() + ']');
+            }
 
             done.set(true);
 
             fut.get();
             getFut.get();
+
+            for (Integer key : keys) {
+                String val = cache.get(key);
+
+                assertNotNull(val);
+            }
+
+            int onheapSize = cache.localSize(CachePeekMode.ONHEAP);
+            int offheapSize = cache.localSize(CachePeekMode.OFFHEAP);
+            int swapSize = cache.localSize(CachePeekMode.SWAP);
+            int total = cache.localSize();
+            long offheapMem = cache.metrics().getOffHeapAllocatedSize();
+
+            log.info("Cache size [heap=" + onheapSize +
+                ", offheap=" + offheapSize +
+                ", swap=" + swapSize +
+                ", total=" + total +
+                ", offheapMem=" + offheapMem +  ']');
+
+            assertTrue(total > 0);
+
+            assertEquals(onheapSize + offheapSize + swapSize, total);
+
+            if (memMode == CacheMemoryMode.OFFHEAP_TIERED)
+                assertEquals(0, onheapSize);
+            else
+                assertEquals(MAX_HEAP_SIZE, onheapSize);
+
+            if (swap) {
+                assertTrue(swapSize > 0);
+                assertTrue(offheapMem <= OFFHEAP_MEM);
+            }
+            else
+                assertEquals(0, swapSize);
         }
         finally {
             done.set(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index d9510e6..1fef4d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -803,6 +803,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public boolean offheapSwapEvict(byte[] vb, GridCacheVersion evictVer, GridCacheVersion obsoleteVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public CacheObject unswap(boolean needVal) throws IgniteCheckedException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
index 4bda4f4..d3241f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     private static final Random RAND = new Random();
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -86,7 +86,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      *
@@ -551,6 +551,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -587,6 +591,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -622,6 +630,10 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
index d1a1b20..f7388e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -39,7 +39,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -91,7 +91,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index 03fcd4a..032c98f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -882,6 +882,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -921,6 +925,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
                 @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                     evictCnt.incrementAndGet();
                 }
+
+                @Override public boolean removedEvicted() {
+                    return true;
+                }
             };
 
             map = newMap();
@@ -957,6 +965,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
             @Override public void onEvict(int part, int hash, byte[] k, byte[] v) {
                 evictCnt.incrementAndGet();
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1009,6 +1021,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();
@@ -1072,6 +1088,10 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
                 evicted.set(key);
             }
+
+            @Override public boolean removedEvicted() {
+                return true;
+            }
         };
 
         map = newMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index e758246..58ad494 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
index 43fdb34..0e36f3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMapSelfTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapFactory;
  */
 public class GridUnsafeMapSelfTest extends GridOffHeapMapAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictLsnr);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
index ed37306..4064482 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapMapPerformanceAbstractTest.java
@@ -42,7 +42,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
         new HashMap<>(LOAD_CNT);
 
     /** Unsafe map. */
-    private GridOffHeapMap<String> map;
+    private GridOffHeapMap map;
 
     /** */
     protected float load = 0.75f;
@@ -94,7 +94,7 @@ public abstract class GridOffHeapMapPerformanceAbstractTest extends GridCommonAb
     /**
      * @return New map.
      */
-    protected abstract <K> GridOffHeapMap<K> newMap();
+    protected abstract GridOffHeapMap newMap();
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
index 1486a9c..af691b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/unsafe/GridUnsafeMapPerformanceTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.offheap.GridOffHeapMapPerformanceAbstract
  */
 public class GridUnsafeMapPerformanceTest extends GridOffHeapMapPerformanceAbstractTest {
     /** {@inheritDoc} */
-    @Override protected <K> GridOffHeapMap<K> newMap() {
+    @Override protected GridOffHeapMap newMap() {
         return GridOffHeapMapFactory.unsafeMap(concurrency, load, initCap, mem, lruStripes, evictClo);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index b02b37e..be644e2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -32,7 +32,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -55,6 +54,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  * Multi-threaded tests for cache queries.
@@ -111,7 +111,7 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setBackups(1);
 
@@ -139,6 +139,11 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
         return cacheCfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return DURATION + 60_000;
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9222164/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
index 909fd74..d7d2b5a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
@@ -22,11 +22,6 @@ package org.apache.ignite.internal.processors.cache;
  */
 public class IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest extends IgniteCacheQueryOffheapMultiThreadedSelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        fail("IGNITE-959");
-    }
-
-    /** {@inheritDoc} */
     @Override protected boolean evictsEnabled() {
         return true;
     }


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

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


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

Branch: refs/heads/ignite-1282
Commit: 40ae36441d863e0295d7d16a97ddbb910f4059dc
Parents: 101be8a 4ae61ce
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 15:02:28 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 15:02:28 2015 +0300

----------------------------------------------------------------------

----------------------------------------------------------------------



[11/17] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-1282
Commit: 4ae61ce9e992f8930432e5e3f85b293cb9784dd3
Parents: 9601736 72e9973
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Sep 10 13:06:39 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 10 13:06:39 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |   6 +-
 .../dotnet/PlatformDotNetConfiguration.java     | 119 +++++++++
 .../PlatformDotNetPortableConfiguration.java    | 228 +++++++++++++++++
 ...PlatformDotNetPortableTypeConfiguration.java | 248 +++++++++++++++++++
 .../dotnet/PlatformDotNetConfiguration.java     | 119 ---------
 .../PlatformDotNetPortableConfiguration.java    | 228 -----------------
 ...PlatformDotNetPortableTypeConfiguration.java | 248 -------------------
 .../PlatformDotNetConfigurationClosure.java     |   1 -
 .../dotnet/PlatformDotNetConfigurationEx.java   |   1 -
 .../Config/Compute/compute-standalone.xml       |   8 +-
 .../Config/cache-portables.xml                  |   4 +-
 .../Config/cache-query.xml                      |   4 +-
 .../native-client-test-cache-affinity.xml       |   6 +-
 13 files changed, 609 insertions(+), 611 deletions(-)
----------------------------------------------------------------------



[02/17] ignite git commit: Merge remote-tracking branch 'origin/master' into ignite-971

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master' into ignite-971


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

Branch: refs/heads/ignite-1282
Commit: 1b51ec3dcd45ace7741b9493ee2241a0fe39e8b3
Parents: b922216 0bd62ef
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 10 09:49:13 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 10 09:49:13 2015 +0300

----------------------------------------------------------------------
 examples/config/platform/cpp/example-cache.xml  |   87 +
 examples/platform/cpp/Makefile.am               |   39 +
 examples/platform/cpp/README.txt                |   42 +
 examples/platform/cpp/configure.ac              |   30 +
 examples/platform/cpp/include/Makefile.am       |   21 +
 .../cpp/include/ignite/examples/address.h       |  109 +
 .../cpp/include/ignite/examples/organization.h  |  111 +
 .../platform/cpp/project/vs/ignite-examples.sln |   19 +
 .../cpp/project/vs/ignite-examples.vcxproj      |  107 +
 .../project/vs/ignite-examples.vcxproj.filters  |   30 +
 examples/platform/cpp/src/putgetexample.cpp     |  126 +
 .../portable/PortableClassDescriptor.java       |   79 +-
 .../internal/portable/PortableContext.java      |  120 +-
 .../internal/portable/PortableWriterExImpl.java |   92 +-
 .../processors/cache/IgniteCacheProxy.java      |   27 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |    5 +-
 .../cache/query/GridCacheQueryAdapter.java      |   43 +-
 .../query/GridCacheQueryFutureAdapter.java      |    9 +-
 .../cache/query/GridCacheQueryManager.java      |   11 +-
 .../query/GridCacheQueryMetricsAdapter.java     |  125 +-
 .../callback/PlatformCallbackGateway.java       |    5 +-
 .../callback/PlatformCallbackUtils.java         |    3 +-
 .../processors/query/GridQueryProcessor.java    |   50 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 .../GridPortableMarshallerSelfTest.java         |   68 +-
 .../portable/GridPortableWildcardsSelfTest.java |   60 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |  279 +-
 modules/platform/licenses/apache-2.0.txt        |  202 ++
 modules/platform/pom.xml                        |   14 +
 modules/platform/src/main/cpp/README.txt        |  103 +
 .../platform/src/main/cpp/common/configure.ac   |    2 +-
 .../src/main/cpp/common/ignite-common.pc.in     |    2 +-
 .../cpp/common/include/ignite/common/exports.h  |    8 +-
 .../cpp/common/include/ignite/common/java.h     |   20 +-
 .../main/cpp/common/project/vs/common.vcxproj   |    2 +-
 .../src/main/cpp/common/src/exports.cpp         |   16 +-
 .../platform/src/main/cpp/common/src/java.cpp   |   53 +-
 .../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                   |  283 +-
 .../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 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |    2 +-
 .../Common/IgniteException.cs                   |    2 +-
 .../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  |  640 +++-
 .../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/Common/DelegateTypeDescriptor.cs       |  314 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    4 +-
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 -
 .../Impl/Common/IFutureConverter.cs             |    2 +
 .../Impl/Common/IFutureInternal.cs              |    1 +
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../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 ++
 .../Impl/Handle/HandleRegistry.cs               |    4 +-
 .../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    |  492 +++
 .../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              |    3 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Impl/Memory/PlatformMemory.cs               |    1 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    5 +-
 .../Impl/Memory/PlatformMemoryPool.cs           |    1 +
 .../Impl/Memory/PlatformMemoryStream.cs         |    4 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    3 +-
 .../Impl/Memory/PlatformPooledMemory.cs         |    2 +-
 .../Impl/Memory/PlatformRawMemory.cs            |    1 +
 .../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/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                  |    3 +-
 .../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  |   37 +-
 .../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 +
 .../platform/PlatformProcessorImpl.java         |    2 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  162 +-
 .../Cache/CacheAbstractTest.cs                  | 3321 ++++++++++++++++++
 .../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/IgnitionTest.cs    |   30 -
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../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                  |    2 +-
 .../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      |   15 +-
 .../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         |  119 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 parent/pom.xml                                  |   14 +
 547 files changed, 102629 insertions(+), 1254 deletions(-)
----------------------------------------------------------------------



[16/17] ignite git commit: Merge branch 'master' into ignite-1282

Posted by vo...@apache.org.
Merge branch 'master' into ignite-1282


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

Branch: refs/heads/ignite-1282
Commit: b7a24bfcafd452a3c2ef99b5cf7e0004d4e5110d
Parents: 87fd9ac 40ae364
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 15:03:29 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 15:03:29 2015 +0300

----------------------------------------------------------------------
 examples/config/platform/cpp/example-cache.xml  |  87 ---------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/apache-license-gen/pom.xml              |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../configuration/CacheConfiguration.java       |  41 +---
 .../ignite/configuration/TopologyValidator.java |  24 ++-
 .../internal/portable/PortableReaderExImpl.java |   3 -
 .../processors/cache/CacheMetricsImpl.java      |   2 +-
 .../processors/cache/GridCacheEntryEx.java      |  13 ++
 .../cache/GridCacheEvictionManager.java         |  18 +-
 .../processors/cache/GridCacheMapEntry.java     |  45 ++++-
 .../cache/GridCacheSwapEntryImpl.java           |  24 +--
 .../processors/cache/GridCacheSwapManager.java  | 168 +++++++++++++---
 .../processors/cache/GridCacheUtils.java        |  21 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +-
 .../offheap/GridOffHeapProcessor.java           |  25 ++-
 .../util/offheap/GridOffHeapEvictListener.java  |   5 +
 .../internal/util/offheap/GridOffHeapMap.java   |  13 +-
 .../util/offheap/GridOffHeapMapFactory.java     |  28 +--
 .../util/offheap/GridOffHeapPartitionedMap.java |  11 ++
 .../util/offheap/unsafe/GridUnsafeMap.java      | 128 +++++++++----
 .../unsafe/GridUnsafePartitionedMap.java        |   9 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java | 168 ++++++++++++++--
 .../GridPortableMarshallerSelfTest.java         |   1 -
 .../cache/CacheSwapUnswapGetTest.java           |  85 ++++++++-
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java | 191 +++++++------------
 .../offheap/GridOffHeapMapAbstractSelfTest.java |  16 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |  20 ++
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../offheap/unsafe/GridUnsafeMapSelfTest.java   |   2 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   4 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   9 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jms11/pom.xml                           |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/kafka/pom.xml                           |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/log4j2/pom.xml                          |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/platform/pom.xml                        |   2 +-
 .../platform/src/main/cpp/common/configure.ac   |   2 +-
 .../src/main/cpp/core-test/configure.ac         |   2 +-
 modules/platform/src/main/cpp/core/configure.ac |   2 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 modules/yarn/pom.xml                            |   2 +-
 modules/zookeeper/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 86 files changed, 823 insertions(+), 477 deletions(-)
----------------------------------------------------------------------



[04/17] ignite git commit: Merge remote-tracking branch 'origin/master' into ignite-971

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master' into ignite-971


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

Branch: refs/heads/ignite-1282
Commit: 1feff74f4df3a903dcce411d727606672bc36f5b
Parents: 4a3b024 45df2a7
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 10 10:46:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 10 10:46:38 2015 +0300

----------------------------------------------------------------------
 examples/platform/cpp/Makefile.am               |  39 ------
 examples/platform/cpp/README.txt                |  42 -------
 examples/platform/cpp/configure.ac              |  30 -----
 examples/platform/cpp/include/Makefile.am       |  21 ----
 .../cpp/include/ignite/examples/address.h       | 109 ----------------
 .../cpp/include/ignite/examples/organization.h  | 111 ----------------
 .../platform/cpp/project/vs/ignite-examples.sln |  19 ---
 .../cpp/project/vs/ignite-examples.vcxproj      | 107 ----------------
 .../project/vs/ignite-examples.vcxproj.filters  |  30 -----
 examples/platform/cpp/src/putgetexample.cpp     | 126 -------------------
 .../configuration/IgniteConfiguration.java      |  22 ----
 .../configuration/PlatformConfiguration.java    |  25 ----
 .../apache/ignite/internal/IgniteKernal.java    |   8 +-
 .../platform/PlatformConfiguration.java         |  25 ++++
 .../dotnet/PlatformDotNetConfiguration.java     |   2 +-
 .../platform/PlatformProcessorImpl.java         |   3 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   6 +-
 .../PlatformDotNetConfigurationClosure.java     |   6 +-
 .../platform/cpp/PlatformCppConfiguration.java  |   2 +-
 19 files changed, 39 insertions(+), 694 deletions(-)
----------------------------------------------------------------------



[08/17] ignite git commit: IGNITE-802: reworked GridCachePartitionedQueueEntryMoveSelfTest.testQueue

Posted by vo...@apache.org.
IGNITE-802: reworked GridCachePartitionedQueueEntryMoveSelfTest.testQueue


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

Branch: refs/heads/ignite-1282
Commit: ec5c795aa523cc48c292cfb09e422edcd8a1a42b
Parents: d96e0d2
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 10 12:18:44 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 10 12:18:44 2015 +0300

----------------------------------------------------------------------
 ...dCachePartitionedQueueEntryMoveSelfTest.java | 191 +++++++------------
 1 file changed, 66 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5c795a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
index 4d92b88..1d225a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.datastructures.partitioned;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
@@ -30,18 +29,15 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteCollectionAbstractTest;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.GridTestNode;
 import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -52,11 +48,6 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  * Cache queue test with changing topology.
  */
 public class GridCachePartitionedQueueEntryMoveSelfTest extends IgniteCollectionAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-802");
-    }
-
     /** Queue capacity. */
     private static final int QUEUE_CAP = 5;
 
@@ -66,9 +57,6 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends IgniteCollection
     /** Backups count. */
     private static final int BACKUP_CNT = 1;
 
-    /** Node ID to set manually on node startup. */
-    private UUID nodeId;
-
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return GRID_CNT;
@@ -98,116 +86,93 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends IgniteCollection
         return colCfg;
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        if (nodeId != null) {
-            cfg.setNodeId(nodeId);
-
-            nodeId = null;
-        }
-
-        return cfg;
-    }
-
     /**
      * @throws Exception If failed.
      */
     public void testQueue() throws Exception {
-        try {
-            startGrids(GRID_CNT);
-
-            final String queueName = "queue-name-" + UUID.randomUUID();
+        final String queueName = "queue-test-name";
 
-            System.out.println(U.filler(20, '\n'));
+        System.out.println(U.filler(20, '\n'));
 
-            final CountDownLatch latch1 = new CountDownLatch(1);
-            //final CountDownLatch latch2 = new CountDownLatch(1);
+        final CountDownLatch latch1 = new CountDownLatch(1);
+        final CountDownLatch latch2 = new CountDownLatch(1);
 
-            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
-                @Override public Void call() {
-                    Ignite ignite = grid(0);
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws IgniteInterruptedCheckedException {
+                Ignite ignite = grid(0);
 
-                    IgniteQueue<Integer> queue = ignite.queue(queueName,
-                        QUEUE_CAP,
-                        config(true));
+                IgniteQueue<Integer> queue = ignite.queue(queueName, QUEUE_CAP, config(true));
 
-                    for (int i = 0; i < QUEUE_CAP * 2; i++) {
-                        if (i == QUEUE_CAP) {
-                            latch1.countDown();
+                for (int i = 0; i < QUEUE_CAP * 2; i++) {
+                    if (i == QUEUE_CAP) {
+                        latch1.countDown();
 
-                            //U.await(latch2);
-                        }
-
-                        try {
-                            info(">>> Putting value: " + i);
+                        U.await(latch2);
+                    }
 
-                            queue.put(i);
+                    try {
+                        info(">>> Putting value: " + i);
 
-                            info(">>> Value is in queue: " + i);
-                        }
-                        catch (Error | RuntimeException e) {
-                            error("Failed to put value: " + i, e);
+                        queue.put(i);
 
-                            throw e;
-                        }
+                        info(">>> Value is in queue: " + i);
                     }
+                    catch (Error | RuntimeException e) {
+                        error("Failed to put value: " + i, e);
 
-                    return null;
+                        throw e;
+                    }
                 }
-            });
 
-            latch1.await();
+                return null;
+            }
+        });
 
-            startAdditionalNodes(BACKUP_CNT + 2, queueName);
+        latch1.await();
 
-            System.out.println(U.filler(20, '\n'));
+        startAdditionalNodes(BACKUP_CNT + 2, queueName);
 
-            //latch2.countDown();
+        System.out.println(U.filler(20, '\n'));
 
-            IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() {
-                @Override public Void call() throws IgniteCheckedException {
-                    Ignite ignite = grid(GRID_CNT);
+        latch2.countDown();
 
-                    IgniteQueue<Integer> queue = ignite.queue(queueName, Integer.MAX_VALUE, config(true));
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws IgniteCheckedException {
+                Ignite ignite = grid(GRID_CNT);
 
-                    int cnt = 0;
+                IgniteQueue<Integer> queue = ignite.queue(queueName, QUEUE_CAP, config(true));
 
-                    do {
-                        try {
-                            Integer i = queue.poll();
+                int cnt = 0;
 
-                            if (i != null) {
-                                info(">>> Polled value: " + cnt);
+                do {
+                    try {
+                        Integer i = queue.poll();
 
-                                cnt++;
-                            }
-                            else {
-                                info(">>> Waiting for value...");
+                        if (i != null) {
+                            info(">>> Polled value: " + cnt);
 
-                                U.sleep(2000);
-                            }
+                            cnt++;
                         }
-                        catch (Error | RuntimeException e) {
-                            error("Failed to poll value.", e);
+                        else {
+                            info(">>> Waiting for value...");
 
-                            throw e;
+                            U.sleep(2000);
                         }
                     }
-                    while (cnt < QUEUE_CAP * 2);
+                    catch (Error | RuntimeException e) {
+                        error("Failed to poll value.", e);
 
-                    return null;
+                        throw e;
+                    }
                 }
-            });
+                while (cnt < QUEUE_CAP * 2);
 
-            fut1.get();
-            fut2.get();
-        }
-        finally {
-            stopAllGrids();
-        }
+                return null;
+            }
+        });
+
+        fut1.get();
+        fut2.get();
     }
 
     /**
@@ -218,51 +183,27 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends IgniteCollection
      * @throws Exception If failed.
      */
     private void startAdditionalNodes(int cnt, String queueName) throws Exception {
-        AffinityFunction aff = jcache(0).getConfiguration(CacheConfiguration.class).getAffinity();
-        AffinityKeyMapper mapper = jcache(0).getConfiguration(CacheConfiguration.class).getAffinityMapper();
-
-        assertNotNull(aff);
-        assertNotNull(mapper);
-
-        int part = aff.partition(mapper.affinityKey(queueName));
+        IgniteQueue queue = ignite(0).queue(queueName, 0, null);
 
-        Collection<ClusterNode> nodes = grid(0).cluster().nodes();
+        CacheConfiguration cCfg = getQueueCache(queue);
 
-        Collection<ClusterNode> aff0 = ignite(0).affinity(null).mapKeyToPrimaryAndBackups(queueName);
-        Collection<ClusterNode> aff1 = nodes(aff, part, nodes);
+        Collection<ClusterNode> aff1 = ignite(0).affinity(cCfg.getName()).mapKeyToPrimaryAndBackups(queueName);
 
-        assertEquals(new ArrayList<>(aff0), new ArrayList<>(aff1));
+        for (int i = 0, id = GRID_CNT; i < cnt; i++) {
+            startGrid(id++);
 
-        Collection<ClusterNode> aff2;
-        Collection<ClusterNode> tmpNodes;
+            awaitPartitionMapExchange();
 
-        int retries = 10000;
+            Collection<ClusterNode> aff2 = ignite(0).affinity(cCfg.getName()).mapKeyToPrimaryAndBackups(queueName);
 
-        do {
-            tmpNodes = new ArrayList<>(cnt);
+            if (!aff1.iterator().next().equals(aff2.iterator().next())) {
+                info("Moved queue to new primary node [oldAff=" + aff1 + ", newAff=" + aff2 + ']');
 
-            for (int i = 0; i < cnt; i++)
-                tmpNodes.add(new GridTestNode(UUID.randomUUID()));
-
-            aff2 = nodes(aff, part, F.concat(true, tmpNodes, nodes));
-
-            if (retries-- < 0)
-                throw new IgniteCheckedException("Failed to find node IDs to change current affinity mapping.");
+                return;
+            }
         }
-        while (F.containsAny(aff1, aff2));
-
-        int i = GRID_CNT;
-
-        // Start several additional grids.
-        for (UUID id : F.nodeIds(tmpNodes)) {
-            nodeId = id;
-
-            startGrid(i++);
-        }
-
-        aff2 = ignite(0).affinity(null).mapKeyToPrimaryAndBackups(queueName);
 
-        assertFalse("Unexpected affinity [aff1=" + aff1 + ", aff2=" + aff2 + ']', F.containsAny(aff1, aff2));
+        throw new IgniteCheckedException("Unable to move the queue to a new primary node");
     }
 
     /**


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

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


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

Branch: refs/heads/ignite-1282
Commit: 72e9973df3b12deeaeca0a0050aa0d517d08ae41
Parents: d96e0d2 9abbe90
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 11:53:32 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 11:53:32 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/apache-license-gen/pom.xml              |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../processors/cache/CacheMetricsImpl.java      |   2 +-
 .../processors/cache/GridCacheEntryEx.java      |  13 ++
 .../cache/GridCacheEvictionManager.java         |  18 +-
 .../processors/cache/GridCacheMapEntry.java     |  45 ++++-
 .../cache/GridCacheSwapEntryImpl.java           |  24 +--
 .../processors/cache/GridCacheSwapManager.java  | 168 +++++++++++++++----
 .../processors/cache/GridCacheUtils.java        |  21 +--
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +-
 .../offheap/GridOffHeapProcessor.java           |  25 ++-
 .../util/offheap/GridOffHeapEvictListener.java  |   5 +
 .../internal/util/offheap/GridOffHeapMap.java   |  13 +-
 .../util/offheap/GridOffHeapMapFactory.java     |  28 ++--
 .../util/offheap/GridOffHeapPartitionedMap.java |  11 ++
 .../util/offheap/unsafe/GridUnsafeMap.java      | 128 +++++++++-----
 .../unsafe/GridUnsafePartitionedMap.java        |   9 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/CacheSwapUnswapGetTest.java           |  85 +++++++++-
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 .../offheap/GridOffHeapMapAbstractSelfTest.java |  16 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |  20 +++
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 .../offheap/unsafe/GridUnsafeMapSelfTest.java   |   2 +-
 .../GridOffHeapMapPerformanceAbstractTest.java  |   4 +-
 .../unsafe/GridUnsafeMapPerformanceTest.java    |   2 +-
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   4 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   9 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jms11/pom.xml                           |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/kafka/pom.xml                           |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/log4j2/pom.xml                          |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/platform/pom.xml                        |   2 +-
 .../platform/src/main/cpp/common/configure.ac   |   2 +-
 .../src/main/cpp/core-test/configure.ac         |   2 +-
 modules/platform/src/main/cpp/core/configure.ac |   2 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 modules/yarn/pom.xml                            |   2 +-
 modules/zookeeper/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 79 files changed, 584 insertions(+), 201 deletions(-)
----------------------------------------------------------------------



[14/17] ignite git commit: Merge branch 'ignite-1.4'

Posted by vo...@apache.org.
Merge branch 'ignite-1.4'


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

Branch: refs/heads/ignite-1282
Commit: 101be8afe4edfa48ffcbf29b5536160e45369647
Parents: 72e9973 071586e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 15:01:56 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 15:01:56 2015 +0300

----------------------------------------------------------------------
 examples/config/platform/cpp/example-cache.xml  |  87 ---------
 .../configuration/CacheConfiguration.java       |  41 +---
 .../ignite/configuration/TopologyValidator.java |  24 ++-
 .../internal/portable/PortableReaderExImpl.java |   3 -
 ...idPortableMarshallerCtxDisabledSelfTest.java | 168 ++++++++++++++--
 .../GridPortableMarshallerSelfTest.java         |   1 -
 ...dCachePartitionedQueueEntryMoveSelfTest.java | 191 +++++++------------
 7 files changed, 239 insertions(+), 276 deletions(-)
----------------------------------------------------------------------



[13/17] ignite git commit: Removed platform CPP example ocnfig.

Posted by vo...@apache.org.
Removed platform CPP example ocnfig.


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

Branch: refs/heads/ignite-1282
Commit: 071586e26d55e3f0af4e8c4fc42d407f30ad58d8
Parents: d36db7b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 15:01:33 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 15:01:33 2015 +0300

----------------------------------------------------------------------
 examples/config/platform/cpp/example-cache.xml | 87 ---------------------
 1 file changed, 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/071586e2/examples/config/platform/cpp/example-cache.xml
----------------------------------------------------------------------
diff --git a/examples/config/platform/cpp/example-cache.xml b/examples/config/platform/cpp/example-cache.xml
deleted file mode 100644
index d720a98..0000000
--- a/examples/config/platform/cpp/example-cache.xml
+++ /dev/null
@@ -1,87 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<!--
-    GridGain Spring configuration file to startup grid cache.
-
-    When starting a standalone GridGain node, you need to execute the following command:
-    {IGNITE_HOME}/bin/ignite.{bat|sh} examples/config/example-cache-dotnet.xml
-
-    When starting GridGain from Java IDE, pass path to this file to GridGain:
-    GridGain.start("examples/config/example-cache-dotnet.xml");
--->
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
-
-        <property name="cacheConfiguration">
-            <list>
-                <!--
-                    Partitioned cache example configuration with portable objects enabled.
-                    Used in .NET example that is available only in enterprise edition.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="backups" value="1"/>
-                </bean>
-
-                <!--
-                    Partitioned cache example configuration.
-                    Used in .NET cache store example that is available only in enterprise edition.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="tx"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="backups" value="1"/>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        GridGain provides several options for automatic discovery that can be used
-                        instead os static IP based discovery. For information on all options refer
-                        to our documentation: http://doc.gridgain.org/latest/Automatic+Node+Discovery
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47501</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-</beans>


[12/17] ignite git commit: Topology validator javadoc fix (cherry picked from commit 9601736)

Posted by vo...@apache.org.
Topology validator javadoc fix
(cherry picked from commit 9601736)


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

Branch: refs/heads/ignite-1282
Commit: d36db7bf8683c0641ea809963b95dfa7e10176f6
Parents: 978c4ed
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Sep 10 13:06:12 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 10 13:08:27 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 41 +-------------------
 .../ignite/configuration/TopologyValidator.java | 24 +++++++++++-
 2 files changed, 25 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d36db7bf/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 643804f..9fb56bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1785,26 +1785,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Gets topology validator.
      * <p>
-     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
-     * Topology is always valid in case no topology validator used.
-     * <p>
-     * In case topology is valid for specific cache all operations on this cache are allowed.
-     * <p>
-     * In case topology is not valid for specific cache all update operations on this cache are restricted:
-     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
-     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
+     * See {@link TopologyValidator} for details.
      *
-     * <p>
-     * Usage example
-     * <p>
-     * Following validator allows to put data only in case topology contains exactly 2 nodes:
-     * <pre>{@code
-     * new TopologyValidator() {
-     *    public boolean validate(Collection<ClusterNode> nodes) {
-     *       return nodes.size() == 2;
-     *    }
-     * }
-     * }</pre>
      * @return validator.
      */
     public TopologyValidator getTopologyValidator() {
@@ -1814,26 +1796,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Sets topology validator.
      * <p>
-     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
-     * Topology is always valid in case no topology validator used.
-     * <p>
-     * In case topology is valid for specific cache all operations on this cache are allowed.
-     * <p>
-     * In case topology is not valid for specific cache all update operations on this cache are restricted:
-     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
-     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
-     *
-     * <p>
-     * Usage example
-     * <p>
-     * Following validator allows to put data only in case topology contains exactly 2 nodes:
-     * <pre>{@code
-     * new TopologyValidator() {
-     *    public boolean validate(Collection<ClusterNode> nodes) {
-     *       return nodes.size() == 2;
-     *    }
-     * }
-     * }</pre>
+     * See {@link TopologyValidator} for details.
      *
      * @param topValidator validator.
      * @return {@code this} for chaining.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d36db7bf/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
index 49c06a0..874fc69 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
@@ -19,10 +19,32 @@ package org.apache.ignite.configuration;
 
 import java.io.Serializable;
 import java.util.Collection;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 
 /**
- * Topology validator.
+ * Topology validator checks whether the new topology is valid for specific cache at each topology change.
+ * <p>
+ * Topology is always valid in case no topology validator used.
+ * <p>
+ * In case topology is valid for specific cache all operations on this cache are allowed.
+ * <p>
+ * In case topology is not valid for specific cache all update operations on this cache are restricted:
+ * <ul>
+ * <li>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.</li>
+ * <li>{@link IgniteException} will be thrown at transaction commit attempt.</li>
+ * </ul>
+ * Usage example
+ * <p>
+ * Following validator allows to put data only in case topology contains exactly 2 nodes:
+ * <pre>{@code
+ * new TopologyValidator() {
+ *    public boolean validate(Collection<ClusterNode> nodes) {
+ *       return nodes.size() == 2;
+ *    }
+ * }
+ * }</pre>
  */
 public interface TopologyValidator extends Serializable {
     /**


[09/17] ignite git commit: ignite-1381: fixed unmarshalling of Externalizable objects

Posted by vo...@apache.org.
ignite-1381: fixed unmarshalling of Externalizable objects


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

Branch: refs/heads/ignite-1282
Commit: 978c4edc155ace34fffbc1f28586a4a9d4dde3a6
Parents: ec5c795
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Sep 10 12:22:55 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Sep 10 12:22:55 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableReaderExImpl.java |   3 -
 ...idPortableMarshallerCtxDisabledSelfTest.java | 168 ++++++++++++++++---
 .../GridPortableMarshallerSelfTest.java         |   1 -
 3 files changed, 148 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/978c4edc/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
index 83ccb65..4ad125a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
@@ -2157,9 +2157,6 @@ public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx
                 if (desc == null)
                     throw new PortableInvalidClassException("Unknown type ID: " + typeId);
 
-                // Skip clsName field if any.
-                rawOff += clsNameLen;
-
                 obj = desc.read(this);
 
                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/978c4edc/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index 08dba92..bd9612c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -17,11 +17,20 @@
 
 package org.apache.ignite.internal.portable;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
 import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -50,36 +59,42 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
 
         IgniteUtils.invoke(PortableMarshaller.class, marsh, "setPortableContext", context);
 
-        SimpleObject obj0 = new SimpleObject();
+        SimpleObject simpleObj = new SimpleObject();
 
-        obj0.b = 2;
-        obj0.bArr = new byte[] {2, 3, 4, 5, 5};
-        obj0.c = 'A';
-        obj0.enumVal = TestEnum.D;
-        obj0.objArr = new Object[] {"hello", "world", "from", "me"};
-        obj0.enumArr = new TestEnum[] {TestEnum.C, TestEnum.B};
+        simpleObj.b = 2;
+        simpleObj.bArr = new byte[] {2, 3, 4, 5, 5};
+        simpleObj.c = 'A';
+        simpleObj.enumVal = TestEnum.D;
+        simpleObj.objArr = new Object[] {"hello", "world", "from", "me"};
+        simpleObj.enumArr = new TestEnum[] {TestEnum.C, TestEnum.B};
 
-        byte[] arr = marsh.marshal(obj0);
+        SimpleObject otherObj = new SimpleObject();
 
-        SimpleObject obj2 = marsh.unmarshal(arr, null);
+        otherObj.b = 3;
+        otherObj.bArr = new byte[] {5, 3, 4};
 
-        assertEquals(obj0.b, obj2.b);
-        assertEquals(obj0.c, obj2.c);
-        assertEquals(obj0.enumVal, obj2.enumVal);
+        simpleObj.otherObj = otherObj;
 
-        for (int i = 0; i < obj0.bArr.length; i++)
-            assertEquals(obj0.bArr[i], obj2.bArr[i]);
+        assertEquals(simpleObj, marsh.unmarshal(marsh.marshal(simpleObj), null));
 
-        for (int i = 0; i < obj0.objArr.length; i++)
-            assertEquals(obj0.objArr[i], obj2.objArr[i]);
+        SimplePortable simplePortable = new SimplePortable();
 
-        for (int i = 0; i < obj0.enumArr.length; i++)
-            assertEquals(obj0.enumArr[i], obj2.enumArr[i]);
+        simplePortable.str = "portable";
+        simplePortable.arr = new long[] {100, 200, 300};
+
+        assertEquals(simplePortable, marsh.unmarshal(marsh.marshal(simplePortable), null));
+
+        SimpleExternalizable simpleExtr = new SimpleExternalizable();
+
+        simpleExtr.str = "externalizable";
+        simpleExtr.arr = new long[] {20000, 300000, 400000};
+
+        assertEquals(simpleExtr, marsh.unmarshal(marsh.marshal(simpleExtr), null));
     }
 
     /**
-     * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name
-     * of a portable object.
+     * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name of
+     * a portable object.
      */
     private static class MarshallerContextWithNoStorage extends MarshallerContextAdapter {
         /** */
@@ -124,5 +139,118 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
 
         /** */
         private TestEnum[] enumArr;
+
+        private SimpleObject otherObj;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimpleObject object = (SimpleObject)o;
+
+            if (b != object.b)
+                return false;
+
+            if (c != object.c)
+                return false;
+
+            if (!Arrays.equals(bArr, object.bArr))
+                return false;
+
+            // Probably incorrect - comparing Object[] arrays with Arrays.equals
+            if (!Arrays.equals(objArr, object.objArr))
+                return false;
+
+            if (enumVal != object.enumVal)
+                return false;
+
+            // Probably incorrect - comparing Object[] arrays with Arrays.equals
+            if (!Arrays.equals(enumArr, object.enumArr))
+                return false;
+
+            return !(otherObj != null ? !otherObj.equals(object.otherObj) : object.otherObj != null);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SimplePortable implements PortableMarshalAware {
+        /** */
+        private String str;
+
+        /** */
+        private long[] arr;
+
+        /** {@inheritDoc} */
+        @Override public void writePortable(PortableWriter writer) throws PortableException {
+            writer.writeString("str", str);
+            writer.writeLongArray("longArr", arr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readPortable(PortableReader reader) throws PortableException {
+            str = reader.readString("str");
+            arr = reader.readLongArray("longArr");
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimplePortable that = (SimplePortable)o;
+
+            if (str != null ? !str.equals(that.str) : that.str != null)
+                return false;
+
+            return Arrays.equals(arr, that.arr);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SimpleExternalizable implements Externalizable {
+        /** */
+        private String str;
+
+        /** */
+        private long[] arr;
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeUTF(str);
+            out.writeObject(arr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            str = in.readUTF();
+            arr = (long[])in.readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimpleExternalizable that = (SimpleExternalizable)o;
+
+            if (str != null ? !str.equals(that.str) : that.str != null)
+                return false;
+
+            return Arrays.equals(arr, that.arr);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/978c4edc/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index 4545a58..21fc81c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -34,7 +34,6 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.UUID;


[17/17] ignite git commit: Restored CPP example.

Posted by vo...@apache.org.
Restored CPP example.


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

Branch: refs/heads/ignite-1282
Commit: 836283df0c9f56e1afc2d1f4ac1f4709a3975298
Parents: b7a24bf
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Sep 10 15:05:15 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Sep 10 15:05:15 2015 +0300

----------------------------------------------------------------------
 examples/config/platform/cpp/example-cache.xml | 77 +++++++++++++++++++++
 1 file changed, 77 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/836283df/examples/config/platform/cpp/example-cache.xml
----------------------------------------------------------------------
diff --git a/examples/config/platform/cpp/example-cache.xml b/examples/config/platform/cpp/example-cache.xml
new file mode 100644
index 0000000..beed238
--- /dev/null
+++ b/examples/config/platform/cpp/example-cache.xml
@@ -0,0 +1,77 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Set to true to enable distributed class loading for examples, default is false. -->
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <!--
+                    Partitioned cache example configuration with portable objects enabled.
+                    Used in .NET example that is available only in enterprise edition.
+                -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!--
+                    Partitioned cache example configuration.
+                    Used in .NET cache store example that is available only in enterprise edition.
+                -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="tx"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="backups" value="1"/>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>


[03/17] ignite git commit: ignite-971 Fix offheap to swap eviction.

Posted by vo...@apache.org.
ignite-971 Fix offheap to swap eviction.


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

Branch: refs/heads/ignite-1282
Commit: 4a3b0242c34e580054be2e23bc01560e96519b0b
Parents: 1b51ec3
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 10 10:42:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 10 10:42:01 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheMetricsImpl.java      |  2 +-
 .../processors/cache/GridCacheMapEntry.java     |  4 +--
 .../processors/cache/GridCacheSwapManager.java  |  9 ++----
 .../processors/cache/GridCacheUtils.java        |  6 ++--
 .../util/offheap/unsafe/GridUnsafeMap.java      | 33 ++++++++++++--------
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |  4 +++
 6 files changed, 33 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 1554e07..dfa0217 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -843,7 +843,7 @@ public class CacheMetricsImpl implements CacheMetrics {
         offHeapEvicts.incrementAndGet();
 
         if (delegate != null)
-            delegate.onOffHeapRemove();
+            delegate.onOffHeapEvict();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/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 3fc1b2e..bdf6057 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
@@ -3681,7 +3681,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (F.isEmptyOrNulls(filter)) {
                 synchronized (this) {
                     if (obsoleteVersionExtras() != null)
-                        return false;
+                        return true;
 
                     CacheObject prev = saveValueForIndexUnlocked();
 
@@ -3725,7 +3725,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     synchronized (this) {
                         if (obsoleteVersionExtras() != null)
-                            return false;
+                            return true;
 
                         if (!v.equals(ver))
                             // Version has changed since entry passed the filter. Do it again.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 53b0421..851c827 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -1059,7 +1059,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         checkIteratorQueue();
 
-        boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()),
+        return offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()),
             new IgniteBiPredicate<Long, Integer>() {
                 @Override public boolean apply(Long ptr, Integer len) {
                     GridCacheVersion ver0 = GridCacheOffheapSwapEntry.version(ptr);
@@ -1068,11 +1068,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 }
             }
         );
-
-        if (rmv && cctx.config().isStatisticsEnabled())
-            cctx.cache().metrics0().onOffHeapRemove();
-
-        return rmv;
     }
 
     /**
@@ -1355,7 +1350,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      */
     public void clearOffHeap() {
         if (offheapEnabled)
-            clearOffHeap();
+            initOffHeap();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 919bce6..2d5698a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1032,9 +1032,11 @@ public class GridCacheUtils {
         ctx.swap().unwindOffheapEvicts();
 
         if (ctx.isNear()) {
-            ctx.near().dht().context().evicts().unwind();
+            GridCacheContext dhtCtx = ctx.near().dht().context();
 
-            ctx.swap().unwindOffheapEvicts();
+            dhtCtx.evicts().unwind();
+
+            dhtCtx.swap().unwindOffheapEvicts();
         }
 
         ctx.ttl().expire();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
index ed13fe1..d0435e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
@@ -1022,12 +1022,26 @@ public class GridUnsafeMap implements GridOffHeapMap {
                         }
 
                         if (cur != 0) {
-                            long a;
+                            long qAddr0 = Entry.queueAddress(cur, mem);
 
-                            assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
-                                "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
+                            assert qAddr == qAddr0 : "Queue node address mismatch " +
+                                "[qAddr=" + qAddr + ", entryQueueAddr=" + qAddr + ']';
+
+                            if (evictLsnr != null) {
+                                keyBytes = Entry.readKeyBytes(cur, mem);
+
+                                int keyLen = Entry.readKeyLength(cur, mem);
+                                int valLen = Entry.readValueLength(cur, mem);
+
+                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                            }
 
                             if (rmvEvicted) {
+                                long a;
+
+                                assert qAddr == (a = Entry.queueAddress(cur, mem)) : "Queue node address mismatch " +
+                                    "[qAddr=" + qAddr + ", entryQueueAddr=" + a + ']';
+
                                 long next = Entry.nextAddress(cur, mem);
 
                                 if (prev != 0)
@@ -1047,18 +1061,11 @@ public class GridUnsafeMap implements GridOffHeapMap {
                                 totalCnt.decrement();
                             }
                             else {
-                                if (qAddr != 0) {
-                                    boolean clear = Entry.clearQueueAddress(cur, qAddr, mem);
+                                boolean clear = Entry.clearQueueAddress(cur, qAddr, mem);
 
-                                    assert clear;
-                                }
+                                assert clear;
 
-                                keyBytes = Entry.readKeyBytes(cur, mem);
-
-                                int keyLen = Entry.readKeyLength(cur, mem);
-                                int valLen = Entry.readValueLength(cur, mem);
-
-                                valBytes = mem.readBytes(cur + HEADER + keyLen, valLen);
+                                relSize = Entry.size(cur, mem);
                             }
                         }
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a3b0242/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 8f453a5..eb0880a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -249,6 +249,10 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
             assertEquals(ENTRY_CNT - i - 1, cache.localSize(CachePeekMode.ONHEAP));
         }
 
+        log.info("Cache size [onheap=" + cache.localSize(CachePeekMode.ONHEAP) +
+            ", offheap=" + cache.localSize(CachePeekMode.OFFHEAP) +
+            ", swap=" + cache.localSize(CachePeekMode.SWAP) + ']');
+
         // Ensure that part of entries located in off-heap memory and part is swapped.
         assertEquals(0, cache.localSize(CachePeekMode.ONHEAP));
         assertTrue(cache.localSize(CachePeekMode.OFFHEAP) > 0);


[05/17] ignite git commit: Disabled test with correct JIRA issue.

Posted by vo...@apache.org.
Disabled test with correct JIRA issue.


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

Branch: refs/heads/ignite-1282
Commit: 94f2362d5c8e82b9d23a6ee361dea0afd4a12e5f
Parents: 1feff74
Author: sboikov <sb...@gridgain.com>
Authored: Thu Sep 10 10:53:15 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Sep 10 10:53:15 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java     | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94f2362d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
index d7d2b5a..2a353bf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
@@ -22,6 +22,11 @@ package org.apache.ignite.internal.processors.cache;
  */
 public class IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest extends IgniteCacheQueryOffheapMultiThreadedSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-971");
+    }
+
+    /** {@inheritDoc} */
     @Override protected boolean evictsEnabled() {
         return true;
     }


[10/17] ignite git commit: Topology validator javadoc fix

Posted by vo...@apache.org.
Topology validator javadoc fix


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

Branch: refs/heads/ignite-1282
Commit: 96017363eea8071df8bf53ead70c6c7d2c26b4ed
Parents: 9abbe90
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Sep 10 13:06:12 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Sep 10 13:06:12 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 41 +-------------------
 .../ignite/configuration/TopologyValidator.java | 24 +++++++++++-
 2 files changed, 25 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/96017363/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 643804f..9fb56bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1785,26 +1785,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Gets topology validator.
      * <p>
-     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
-     * Topology is always valid in case no topology validator used.
-     * <p>
-     * In case topology is valid for specific cache all operations on this cache are allowed.
-     * <p>
-     * In case topology is not valid for specific cache all update operations on this cache are restricted:
-     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
-     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
+     * See {@link TopologyValidator} for details.
      *
-     * <p>
-     * Usage example
-     * <p>
-     * Following validator allows to put data only in case topology contains exactly 2 nodes:
-     * <pre>{@code
-     * new TopologyValidator() {
-     *    public boolean validate(Collection<ClusterNode> nodes) {
-     *       return nodes.size() == 2;
-     *    }
-     * }
-     * }</pre>
      * @return validator.
      */
     public TopologyValidator getTopologyValidator() {
@@ -1814,26 +1796,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Sets topology validator.
      * <p>
-     * Topology validator checks whether the new topology is valid for specific cache at each topology change.
-     * Topology is always valid in case no topology validator used.
-     * <p>
-     * In case topology is valid for specific cache all operations on this cache are allowed.
-     * <p>
-     * In case topology is not valid for specific cache all update operations on this cache are restricted:
-     * <p>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.
-     * <p>{@link IgniteException} will be thrown at transaction commit attempt.
-     *
-     * <p>
-     * Usage example
-     * <p>
-     * Following validator allows to put data only in case topology contains exactly 2 nodes:
-     * <pre>{@code
-     * new TopologyValidator() {
-     *    public boolean validate(Collection<ClusterNode> nodes) {
-     *       return nodes.size() == 2;
-     *    }
-     * }
-     * }</pre>
+     * See {@link TopologyValidator} for details.
      *
      * @param topValidator validator.
      * @return {@code this} for chaining.

http://git-wip-us.apache.org/repos/asf/ignite/blob/96017363/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
index 49c06a0..874fc69 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
@@ -19,10 +19,32 @@ package org.apache.ignite.configuration;
 
 import java.io.Serializable;
 import java.util.Collection;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 
 /**
- * Topology validator.
+ * Topology validator checks whether the new topology is valid for specific cache at each topology change.
+ * <p>
+ * Topology is always valid in case no topology validator used.
+ * <p>
+ * In case topology is valid for specific cache all operations on this cache are allowed.
+ * <p>
+ * In case topology is not valid for specific cache all update operations on this cache are restricted:
+ * <ul>
+ * <li>{@link CacheException} will be thrown at update operations (put, remove, etc) attempt.</li>
+ * <li>{@link IgniteException} will be thrown at transaction commit attempt.</li>
+ * </ul>
+ * Usage example
+ * <p>
+ * Following validator allows to put data only in case topology contains exactly 2 nodes:
+ * <pre>{@code
+ * new TopologyValidator() {
+ *    public boolean validate(Collection<ClusterNode> nodes) {
+ *       return nodes.size() == 2;
+ *    }
+ * }
+ * }</pre>
  */
 public interface TopologyValidator extends Serializable {
     /**


[06/17] ignite git commit: 1.5.0-SNAPSHOT

Posted by vo...@apache.org.
1.5.0-SNAPSHOT


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

Branch: refs/heads/ignite-1282
Commit: 9abbe90b9adb642e8506d1d3dbe1fa0e0655ea7b
Parents: 94f2362
Author: Ignite Teamcity <ig...@apache.org>
Authored: Thu Sep 10 11:19:13 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Thu Sep 10 11:19:13 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                                 | 2 +-
 modules/aop/pom.xml                                              | 2 +-
 modules/apache-license-gen/pom.xml                               | 2 +-
 modules/aws/pom.xml                                              | 2 +-
 modules/clients/pom.xml                                          | 2 +-
 modules/cloud/pom.xml                                            | 2 +-
 modules/codegen/pom.xml                                          | 2 +-
 modules/core/pom.xml                                             | 2 +-
 modules/core/src/main/resources/ignite.properties                | 2 +-
 modules/extdata/p2p/pom.xml                                      | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml               | 2 +-
 modules/extdata/uri/pom.xml                                      | 2 +-
 modules/gce/pom.xml                                              | 2 +-
 modules/geospatial/pom.xml                                       | 2 +-
 modules/hadoop/pom.xml                                           | 2 +-
 modules/hibernate/pom.xml                                        | 2 +-
 modules/indexing/pom.xml                                         | 2 +-
 modules/jcl/pom.xml                                              | 2 +-
 modules/jms11/pom.xml                                            | 2 +-
 modules/jta/pom.xml                                              | 2 +-
 modules/kafka/pom.xml                                            | 2 +-
 modules/log4j/pom.xml                                            | 2 +-
 modules/log4j2/pom.xml                                           | 2 +-
 modules/mesos/pom.xml                                            | 2 +-
 modules/platform/pom.xml                                         | 2 +-
 modules/platform/src/main/cpp/common/configure.ac                | 2 +-
 modules/platform/src/main/cpp/core-test/configure.ac             | 2 +-
 modules/platform/src/main/cpp/core/configure.ac                  | 2 +-
 .../main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs    | 4 ++--
 .../src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs     | 4 ++--
 .../Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs  | 4 ++--
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs   | 4 ++--
 modules/rest-http/pom.xml                                        | 2 +-
 modules/scalar-2.10/pom.xml                                      | 2 +-
 modules/scalar/pom.xml                                           | 2 +-
 modules/schedule/pom.xml                                         | 2 +-
 modules/schema-import/pom.xml                                    | 2 +-
 modules/slf4j/pom.xml                                            | 2 +-
 modules/spark-2.10/pom.xml                                       | 2 +-
 modules/spark/pom.xml                                            | 2 +-
 modules/spring/pom.xml                                           | 2 +-
 modules/ssh/pom.xml                                              | 2 +-
 modules/tools/pom.xml                                            | 2 +-
 modules/urideploy/pom.xml                                        | 2 +-
 modules/visor-console-2.10/pom.xml                               | 2 +-
 modules/visor-console/pom.xml                                    | 2 +-
 modules/visor-plugins/pom.xml                                    | 2 +-
 modules/web/pom.xml                                              | 2 +-
 modules/yardstick/pom.xml                                        | 2 +-
 modules/yarn/pom.xml                                             | 2 +-
 modules/zookeeper/pom.xml                                        | 2 +-
 pom.xml                                                          | 2 +-
 52 files changed, 56 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 64e8be9..e4ec73a 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 391f5b8..bb419fa 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index c17f98f..fa6a1e2 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index a0ca6b0..8042d33 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 61f6694..fe8666b 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index b7aa43b..a49625d 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 89cadf5..957049b 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 2f0dde7..e02bb23 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <repositories>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 50725fa..2cf7d15 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.4.0-SNAPSHOT
+ignite.version=1.5.0-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index aca3132..26d1cd2 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index 98d3fcf..08edb5b 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 7613f0c..d9a9297 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 05f043f..1f8de08 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 766114b..39ebdc4 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 839ffb8..c2cb89b 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index a3ae729..8a04ab7 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index e4713b4..d539f6b 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 335b6c8..c94c906 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index dbb385d..47a1495 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jms11</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index c0ce618..78242e7 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 5103a32..89c1550 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index aaf1dca..fe95700 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index 2c0fd94..eca3b75 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j2</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index a652b60..466805c 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index ba5b9d0..ac34436 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-platform</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/main/cpp/common/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/configure.ac b/modules/platform/src/main/cpp/common/configure.ac
index b34d7d8..5cab969 100644
--- a/modules/platform/src/main/cpp/common/configure.ac
+++ b/modules/platform/src/main/cpp/common/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite JNI bridge for C++], [1.4.0], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
+AC_INIT([Apache Ignite JNI bridge for C++], [1.5.0], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/main/cpp/core-test/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/core-test/configure.ac b/modules/platform/src/main/cpp/core-test/configure.ac
index 3a1b660..b337fba 100644
--- a/modules/platform/src/main/cpp/core-test/configure.ac
+++ b/modules/platform/src/main/cpp/core-test/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++ Test], [1.4.0], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++ Test], [1.5.0], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/main/cpp/core/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/core/configure.ac b/modules/platform/src/main/cpp/core/configure.ac
index cdd238f..c1657d3 100644
--- a/modules/platform/src/main/cpp/core/configure.ac
+++ b/modules/platform/src/main/cpp/core/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++], [1.4.0], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++], [1.5.0], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 5f2c018..1876e24 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("1.4.0")]
-[assembly: AssemblyFileVersion("1.4.0")]
+[assembly: AssemblyVersion("1.5.0")]
+[assembly: AssemblyFileVersion("1.5.0")]
 
 [assembly: CLSCompliant(true)]
 [assembly: InternalsVisibleTo("Apache.Ignite")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
index ca38ea2..03f7fb9 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")]
 
-[assembly: AssemblyVersion("1.4.0")]
-[assembly: AssemblyFileVersion("1.4.0")]
\ No newline at end of file
+[assembly: AssemblyVersion("1.5.0")]
+[assembly: AssemblyFileVersion("1.5.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
index cb633ed..22d74c9 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@ -45,5 +45,5 @@ using System.Runtime.InteropServices;
 // You can specify all the values or you can default the Build and Revision Numbers 
 // by using the '*' as shown below:
 // [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyVersion("1.4.0")]
-[assembly: AssemblyFileVersion("1.4.0")]
+[assembly: AssemblyVersion("1.5.0")]
+[assembly: AssemblyFileVersion("1.5.0")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 947e18c..1ebcf24 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")]
 
-[assembly: AssemblyVersion("1.4.0")]
-[assembly: AssemblyFileVersion("1.4.0")]
\ No newline at end of file
+[assembly: AssemblyVersion("1.5.0")]
+[assembly: AssemblyFileVersion("1.5.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index c506df6..58eb1ed 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 627f2b4..c2046e9 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index ab03a5d..1443cc1 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 207a97c..3f53df6 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index ef36af9..1e63cae 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 807ed0d..b22ae6c 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 283855e..316b13b 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index f7b2110..95e05d8 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index bf093ef..8d1f918 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 5b407ad..ff3e70d 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index ca0d0f4..26689760 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index df2b0fa..c8fac6e 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index fa541dc..73c7a58 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index dc841bf..b85cd69 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index c0e5b5e..05b4ede 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index d814f47..99a356e 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index eda9429..2f0c5fe 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index c93efa4..2b758dc 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index 3993bd5..9f5bc42 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-zookeeper</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9abbe90b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d026b30..6f9c848 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.4.0-SNAPSHOT</version>
+    <version>1.5.0-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>