You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2016/02/12 00:27:36 UTC

[01/43] ignite git commit: Fixed broken osgi-related Kafka tests.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2249 68d613028 -> 798e570bb


Fixed broken osgi-related Kafka tests.


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

Branch: refs/heads/ignite-2249
Commit: 532b37358de4f5c4143b6692178dca23e37f1fda
Parents: e2be94e
Author: shtykh_roman <rs...@yahoo.com>
Authored: Fri Feb 5 12:32:06 2016 +0900
Committer: shtykh_roman <rs...@yahoo.com>
Committed: Fri Feb 5 12:32:06 2016 +0900

----------------------------------------------------------------------
 modules/osgi-karaf/src/main/resources/features.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/532b3735/modules/osgi-karaf/src/main/resources/features.xml
----------------------------------------------------------------------
diff --git a/modules/osgi-karaf/src/main/resources/features.xml b/modules/osgi-karaf/src/main/resources/features.xml
index 983aeed..584429d 100644
--- a/modules/osgi-karaf/src/main/resources/features.xml
+++ b/modules/osgi-karaf/src/main/resources/features.xml
@@ -154,16 +154,15 @@
 
     <feature name="ignite-kafka" version="${project.version}" description="Apache Ignite :: Kafka">
         <details>
-            <![CDATA[The Apache Ignite Kafka module + dependencies. This module installs the Scala 2.10 library bundle.]]>
+            <![CDATA[The Apache Ignite Kafka module + dependencies. This module installs the Scala 2.1 library bundle.]]>
         </details>
         <feature prerequisite="true">wrap</feature>
-        <bundle start="true" dependency="true">mvn:org.scala-lang/scala-library/${scala210.library.version}</bundle>
-        <bundle start="true" dependency="true">mvn:org.apache.zookeeper/zookeeper/${zookeeper.version}</bundle>
-        <bundle start="true" dependency="true">wrap:mvn:com.101tec/zkclient/${zkclient.version}$Bundle-SymbolicName=zkclient&amp;Bundle-Version=${zkclient.version}&amp;Export-Package=*;-noimport:=true;version=${zkclient.version}</bundle>
+        <bundle start="true" dependency="true">mvn:org.scala-lang/scala-library/${scala211.library.version}</bundle>
         <bundle start="true" dependency="true">wrap:mvn:com.yammer.metrics/metrics-core/${yammer.metrics.core.version}$Bundle-SymbolicName=yammer-metrics-core&amp;Bundle-Version=2.2.0&amp;Export-Package=*;-noimport:=true;version=${yammer.metrics.core.version}</bundle>
         <bundle start="true" dependency="true">wrap:mvn:com.yammer.metrics/metrics-annotation/${yammer.metrics.annotation.version}$Bundle-SymbolicName=yammer-metrics-annotation&amp;Bundle-Version=2.2.0&amp;Export-Package=*;-noimport:=true;version=${yammer.metrics.annotation.version}</bundle>
-        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.kafka-clients/${kafka.clients.bundle.version}</bundle>
-        <bundle start="true" dependency="true">mvn:org.apache.servicemix.bundles/org.apache.servicemix.bundles.kafka_2.10/${kafka.bundle.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:org.apache.kafka/connect-api/${kafka.version}$Bundle-SymbolicName=connect-api&amp;Bundle-Version=${kafka.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:org.apache.kafka/kafka_2.11/${kafka.version}$Bundle-SymbolicName=kafka_2.11&amp;Bundle-Version=${kafka.version}</bundle>
+        <bundle start="true" dependency="true">wrap:mvn:org.apache.kafka/kafka-clients/${kafka.version}$Bundle-SymbolicName=kafka-clients&amp;Bundle-Version=${kafka.version}</bundle>
         <bundle start="true">mvn:org.apache.ignite/ignite-kafka/${project.version}</bundle>
     </feature>
 
@@ -317,6 +316,7 @@
         <details>
             <![CDATA[The Apache Ignite ZooKeeper module + dependencies.]]>
         </details>
+        <bundle start="true" dependency="true">mvn:org.apache.zookeeper/zookeeper/${zookeeper.version}</bundle>
         <bundle start="true" dependency="true">mvn:com.google.guava/guava/${guava16.version}</bundle>
         <bundle start="true" dependency="true">mvn:org.codehaus.jackson/jackson-core-asl/${jackson.version}</bundle>
         <bundle start="true" dependency="true">mvn:org.codehaus.jackson/jackson-mapper-asl/${jackson.version}</bundle>


[30/43] ignite git commit: ignite-2587 Fixed continuous query notifications in offheap mode and BinaryObjectOffheapImpl usage.

Posted by vk...@apache.org.
ignite-2587 Fixed continuous query notifications in offheap mode and BinaryObjectOffheapImpl usage.


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

Branch: refs/heads/ignite-2249
Commit: 4c05fc0254f446ef040f5d22a066a0d4916a589e
Parents: 0b47d5c
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 10 14:07:40 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 10 14:07:40 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheLazyEntry.java        |   3 +
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheMapEntry.java     | 118 +++-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  79 ++-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  85 ++-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  38 +-
 .../cache/query/GridCacheQueryManager.java      |  30 +-
 .../continuous/CacheContinuousQueryHandler.java |   3 +-
 .../CacheContinuousQueryListener.java           |   2 +-
 .../continuous/CacheContinuousQueryManager.java | 120 +++-
 .../continuous/GridContinuousProcessor.java     |  16 +-
 .../IgniteCacheEntryListenerAbstractTest.java   | 454 ++++++++----
 ...cheEntryListenerAtomicOffheapTieredTest.java |  32 +
 ...cheEntryListenerAtomicOffheapValuesTest.java |  32 +
 ...teCacheEntryListenerTxOffheapTieredTest.java |  32 +
 ...teCacheEntryListenerTxOffheapValuesTest.java |  32 +
 .../cache/IgniteCacheEntryListenerTxTest.java   |   1 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |  10 +
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |  33 +
 ...tinuousQueryFailoverTxOffheapTieredTest.java |  32 +
 ...acheContinuousQueryRandomOperationsTest.java | 684 +++++++++++++++++++
 ...ridCacheContinuousQueryAbstractSelfTest.java |  19 +-
 ...eContinuousQueryAtomicOffheapTieredTest.java |  32 +
 ...eContinuousQueryAtomicOffheapValuesTest.java |  32 +
 ...CacheContinuousQueryTxOffheapTieredTest.java |  32 +
 ...CacheContinuousQueryTxOffheapValuesTest.java |  32 +
 .../junits/common/GridCommonAbstractTest.java   |   2 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   8 +
 .../IgniteCacheQuerySelfTestSuite.java          |  14 +
 30 files changed, 1743 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
index 05a6fef..30933e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
@@ -50,6 +50,7 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param cctx Cache context.
      * @param keyObj Key cache object.
      * @param valObj Cache object value.
+     * @param keepBinary Keep binary flag.
      */
     public CacheLazyEntry(GridCacheContext cctx, KeyCacheObject keyObj, CacheObject valObj, boolean keepBinary) {
         this.cctx = cctx;
@@ -61,6 +62,7 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
     /**
      * @param keyObj Key cache object.
      * @param val Value.
+     * @param keepBinary Keep binary flag.
      * @param cctx Cache context.
      */
     public CacheLazyEntry(GridCacheContext cctx, KeyCacheObject keyObj, V val, boolean keepBinary) {
@@ -75,6 +77,7 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param keyObj Key cache object.
      * @param key Key value.
      * @param valObj Cache object
+     * @param keepBinary Keep binary flag.
      * @param val Cache value.
      */
     public CacheLazyEntry(GridCacheContext<K, V> ctx,

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index e875df0..5729959 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1729,10 +1729,10 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return Heap-based object.
      */
     @Nullable public <T> T unwrapTemporary(@Nullable Object obj) {
-        if (!offheapTiered())
+        if (!useOffheapEntry())
             return (T)obj;
 
-        return (T) cacheObjects().unwrapTemporary(this, obj);
+        return (T)cacheObjects().unwrapTemporary(this, obj);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/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 ae40295..9336e0a 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
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.Cache;
@@ -44,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.extras.GridCacheMvccEntryExtr
 import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras;
 import org.apache.ignite.internal.processors.cache.extras.GridCacheTtlEntryExtras;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryListener;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -1122,7 +1124,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             assert newVer != null : "Failed to get write version for tx: " + tx;
 
-            old = (retval || intercept) ? rawGetOrUnmarshalUnlocked(!retval) : this.val;
+            boolean internal = isInternal() || !context().userCache();
+
+            Map<UUID, CacheContinuousQueryListener> lsnrCol =
+                notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null;
+
+            old = (retval || intercept || lsnrCol != null) ?
+                rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : this.val;
 
             if (intercept) {
                 val0 = CU.value(val, cctx, false);
@@ -1206,10 +1214,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     keepBinary);
             }
 
-            if (cctx.isLocal() || cctx.isReplicated() ||
-                (!isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local())))
-                cctx.continuousQueries().onEntryUpdated(key, val, old, isInternal() || !context().userCache(),
-                    partition(), tx.local(), false, updateCntr0, topVer);
+            if (lsnrCol != null) {
+                cctx.continuousQueries().onEntryUpdated(
+                    lsnrCol,
+                    key,
+                    val,
+                    old,
+                    internal,
+                    partition(),
+                    tx.local(),
+                    false,
+                    updateCntr0,
+                    topVer);
+            }
 
             cctx.dataStructures().onEntryUpdated(key, false, keepBinary);
         }
@@ -1304,7 +1321,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             newVer = explicitVer != null ? explicitVer : tx == null ? nextVersion() : tx.writeVersion();
 
-            old = (retval || intercept) ? rawGetOrUnmarshalUnlocked(!retval) : val;
+            boolean internal = isInternal() || !context().userCache();
+
+            Map<UUID, CacheContinuousQueryListener> lsnrCol =
+                notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null;
+
+            old = (retval || intercept || lsnrCol != null) ?
+                rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val;
 
             if (intercept) {
                 entry0 = new CacheLazyEntry(cctx, key, old, keepBinary);
@@ -1388,10 +1411,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     keepBinary);
             }
 
-            if (cctx.isLocal() || cctx.isReplicated() ||
-                (!isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local())))
-                cctx.continuousQueries().onEntryUpdated(key, null, old, isInternal()
-                    || !context().userCache(),partition(), tx.local(), false, updateCntr0, topVer);
+            if (lsnrCol != null) {
+                cctx.continuousQueries().onEntryUpdated(
+                    lsnrCol,
+                    key,
+                    null,
+                    old,
+                    internal,
+                    partition(),
+                    tx.local(),
+                    false,
+                    updateCntr0,
+                    topVer);
+            }
 
             cctx.dataStructures().onEntryUpdated(key, true, keepBinary);
 
@@ -1440,6 +1472,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             return new GridCacheUpdateTxResult(false, null);
     }
 
+    /**
+     * @param tx Transaction.
+     * @return {@code True} if should notify continuous query manager.
+     */
+    private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
+        return cctx.isLocal() ||
+            cctx.isReplicated() ||
+            (!isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public GridTuple3<Boolean, Object, EntryProcessorResult<Object>> innerUpdateLocal(
@@ -1470,7 +1512,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         EntryProcessorResult<Object> invokeRes = null;
 
         synchronized (this) {
-            boolean needVal = retval || intercept || op == GridCacheOperation.TRANSFORM || !F.isEmpty(filter);
+            boolean internal = isInternal() || !context().userCache();
+
+            Map<UUID, CacheContinuousQueryListener> lsnrCol =
+                cctx.continuousQueries().updateListeners(internal, false);
+
+            boolean needVal = retval ||
+                intercept ||
+                op == GridCacheOperation.TRANSFORM ||
+                !F.isEmpty(filter) ||
+                lsnrCol != null;
 
             checkObsolete();
 
@@ -1479,7 +1530,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 unswap(retval);
 
             // Possibly get old value form store.
-            old = needVal ? rawGetOrUnmarshalUnlocked(!retval) : val;
+            old = needVal ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val;
 
             boolean readFromStore = false;
 
@@ -1731,11 +1782,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (res)
                 updateMetrics(op, metrics);
 
-            if (!isNear()) {
+            if (lsnrCol != null) {
                 long updateCntr = nextPartCounter(AffinityTopologyVersion.NONE);
 
-                cctx.continuousQueries().onEntryUpdated(key, val, old, isInternal() || !context().userCache(),
-                    partition(), true, false, updateCntr, AffinityTopologyVersion.NONE);
+                cctx.continuousQueries().onEntryUpdated(
+                    lsnrCol,
+                    key,
+                    val,
+                    old,
+                    internal,
+                    partition(),
+                    true,
+                    false,
+                    updateCntr,
+                    AffinityTopologyVersion.NONE);
             }
 
             cctx.dataStructures().onEntryUpdated(key, op == GridCacheOperation.DELETE, keepBinary);
@@ -1997,8 +2057,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             if (updateCntr != null)
                                 updateCntr0 = updateCntr;
 
-                            cctx.continuousQueries().onEntryUpdated(key, evtVal, prevVal, isInternal()
-                                || !context().userCache(), partition(), primary, false, updateCntr0, topVer);
+                            cctx.continuousQueries().onEntryUpdated(
+                                key,
+                                evtVal,
+                                prevVal,
+                                isInternal() || !context().userCache(),
+                                partition(),
+                                primary,
+                                false,
+                                updateCntr0,
+                                topVer);
                         }
 
                         return new GridCacheUpdateAtomicResult(false,
@@ -2019,7 +2087,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             }
 
             // Prepare old value and value bytes.
-            oldVal = needVal ? rawGetOrUnmarshalUnlocked(!retval) : val;
+            oldVal = needVal ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val;
 
             // Possibly read value from store.
             boolean readFromStore = false;
@@ -2937,7 +3005,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * @return {@code True} if values should be stored off-heap.
      */
-    protected boolean isOffHeapValuesOnly() {
+    protected final boolean isOffHeapValuesOnly() {
         return cctx.config().getMemoryMode() == CacheMemoryMode.OFFHEAP_VALUES;
     }
 
@@ -3236,8 +3304,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 drReplicate(drType, val, ver);
 
                 if (!skipQryNtf) {
-                    cctx.continuousQueries().onEntryUpdated(key, val, null, this.isInternal()
-                        || !this.context().userCache(), this.partition(), true, preload, updateCntr, topVer);
+                    cctx.continuousQueries().onEntryUpdated(
+                        key,
+                        val,
+                        null,
+                        this.isInternal() || !this.context().userCache(),
+                        this.partition(),
+                        true,
+                        preload,
+                        updateCntr,
+                        topVer);
 
                     cctx.dataStructures().onEntryUpdated(key, false, true);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 0fef6f8..f091fc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -822,10 +822,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         Object val = unmarshal(valPtr, !tmp);
 
-        if (val instanceof BinaryObjectOffheapImpl)
-            return (BinaryObjectOffheapImpl)val;
+        if (val instanceof CacheObject)
+            return (CacheObject)val;
 
-        return new CacheObjectImpl(val, null);
+        return toCacheObject(ctx.cacheObjectContext(), val, false);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 6c7bac5..fec61df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -76,6 +76,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSing
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrExpirationInfo;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryListener;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
@@ -1992,6 +1993,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         boolean intercept = ctx.config().getInterceptor() != null;
 
+        boolean initLsnrs = false;
+        Map<UUID, CacheContinuousQueryListener> lsnrs = null;
+        boolean internal = false;
+
         // Avoid iterator creation.
         for (int i = 0; i < keys.size(); i++) {
             KeyCacheObject k = keys.get(i);
@@ -2006,6 +2011,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 if (entry == null)
                     continue;
 
+                if (!initLsnrs) {
+                    internal = entry.isInternal() || !context().userCache();
+
+                    lsnrs = ctx.continuousQueries().updateListeners(internal, false);
+
+                    initLsnrs = true;
+                }
+
                 GridCacheVersion newConflictVer = req.conflictVersion(i);
                 long newConflictTtl = req.conflictTtl(i);
                 long newConflictExpireTime = req.conflictExpireTime(i);
@@ -2034,7 +2047,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     req.invokeArguments(),
                     primary && writeThrough() && !req.skipStore(),
                     !req.skipStore(),
-                    sndPrevVal || req.returnValue(),
+                    lsnrs != null || sndPrevVal || req.returnValue(),
                     req.keepBinary(),
                     expiry,
                     true,
@@ -2061,6 +2074,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 }
 
                 if (dhtFut != null) {
+                    dhtFut.listeners(lsnrs);
+
                     if (updRes.sendToDht()) { // Send to backups even in case of remove-remove scenarios.
                         GridCacheVersionConflictContext<?, ?> conflictCtx = updRes.conflictResolveResult();
 
@@ -2097,10 +2112,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 "[entry=" + entry + ", filter=" + Arrays.toString(req.filter()) + ']');
                     }
                 }
-                else if (!entry.isNear() && updRes.success()) {
-                    ctx.continuousQueries().onEntryUpdated(entry.key(), updRes.newValue(), updRes.oldValue(),
-                        entry.isInternal() || !context().userCache(), entry.partition(), primary, false,
-                        updRes.updateCounter(), topVer);
+                else if (lsnrs != null && updRes.success()) {
+                    ctx.continuousQueries().onEntryUpdated(
+                        lsnrs,
+                        entry.key(),
+                        updRes.newValue(),
+                        updRes.oldValue(),
+                        internal,
+                        entry.partition(),
+                        primary,
+                        false,
+                        updRes.updateCounter(),
+                        topVer);
                 }
 
                 if (hasNear) {
@@ -2275,6 +2298,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             boolean intercept = ctx.config().getInterceptor() != null;
 
+            boolean initLsnrs = false;
+            Map<UUID, CacheContinuousQueryListener> lsnrs = null;
+
             // Avoid iterator creation.
             for (int i = 0; i < entries.size(); i++) {
                 GridDhtCacheEntry entry = entries.get(i);
@@ -2308,6 +2334,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         filteredReaders = F.view(entry.readers(), F.notEqualTo(node.id()));
                     }
 
+                    if (!initLsnrs) {
+                        lsnrs = ctx.continuousQueries().updateListeners(
+                            entry.isInternal() || !context().userCache(),
+                            false);
+
+                        initLsnrs = true;
+                    }
+
                     GridCacheUpdateAtomicResult updRes = entry.innerUpdate(
                         ver,
                         node.id(),
@@ -2317,7 +2351,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         null,
                         /*write-through*/false,
                         /*read-through*/false,
-                        /*retval*/sndPrevVal,
+                        /*retval*/sndPrevVal || lsnrs != null,
                         req.keepBinary(),
                         expiry,
                         /*event*/true,
@@ -2366,6 +2400,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     }
 
                     if (dhtFut != null) {
+                        dhtFut.listeners(lsnrs);
+
                         EntryProcessor<Object, Object, Object> entryProcessor =
                             entryProcessorMap == null ? null : entryProcessorMap.get(entry.key());
 
@@ -2763,6 +2799,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         String taskName = ctx.kernalContext().task().resolveTaskName(req.taskNameHash());
 
+        boolean initLsnrs = false;
+        Map<UUID, CacheContinuousQueryListener> lsnrs = null;
+        boolean internal = false;
+
         for (int i = 0; i < req.size(); i++) {
             KeyCacheObject key = req.key(i);
 
@@ -2785,6 +2825,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         long ttl = req.ttl(i);
                         long expireTime = req.conflictExpireTime(i);
 
+                        if (!initLsnrs) {
+                            internal = entry.isInternal() || !context().userCache();
+
+                            lsnrs = ctx.continuousQueries().updateListeners(internal, false);
+
+                            initLsnrs = true;
+                        }
+
                         GridCacheUpdateAtomicResult updRes = entry.innerUpdate(
                             ver,
                             nodeId,
@@ -2794,7 +2842,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                             op == TRANSFORM ? req.invokeArguments() : null,
                             /*write-through*/false,
                             /*read-through*/false,
-                            /*retval*/false,
+                            /*retval*/lsnrs != null,
                             req.keepBinary(),
                             /*expiry policy*/null,
                             /*event*/true,
@@ -2817,10 +2865,19 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         if (updRes.removeVersion() != null)
                             ctx.onDeferredDelete(entry, updRes.removeVersion());
 
-                        if (updRes.success() && !entry.isNear())
-                            ctx.continuousQueries().onEntryUpdated(entry.key(), updRes.newValue(),
-                                updRes.oldValue(), entry.isInternal() || !context().userCache(), entry.partition(),
-                                false, false, updRes.updateCounter(), req.topologyVersion());
+                        if (lsnrs != null && updRes.success()) {
+                            ctx.continuousQueries().onEntryUpdated(
+                                lsnrs,
+                                entry.key(),
+                                updRes.newValue(),
+                                updRes.oldValue(),
+                                internal,
+                                entry.partition(),
+                                false,
+                                false,
+                                updRes.updateCounter(),
+                                req.topologyVersion());
+                        }
 
                         entry.onUnlock();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 06c8441..58d704d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryListener;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -102,6 +103,9 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
     /** Response count. */
     private volatile int resCnt;
 
+    /** */
+    private Map<UUID, CacheContinuousQueryListener> lsnrs;
+
     /**
      * @param cctx Cache context.
      * @param completionCb Callback to invoke when future is completed.
@@ -136,6 +140,13 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
         waitForExchange = !topLocked;
     }
 
+    /**
+     * @param lsnrs Continuous query listeners.
+     */
+    void listeners(@Nullable Map<UUID, CacheContinuousQueryListener> lsnrs) {
+        this.lsnrs = lsnrs;
+    }
+
     /** {@inheritDoc} */
     @Override public IgniteUuid futureId() {
         return futVer.asGridUuid();
@@ -215,6 +226,8 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
      * @param ttl TTL (optional).
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
+     * @param addPrevVal If {@code true} sends previous value to backups.
+     * @param prevVal Previous value.
      * @param updateCntr Partition update counter.
      */
     public void addWriteEntry(GridDhtCacheEntry entry,
@@ -270,13 +283,22 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
                     addPrevVal,
                     entry.partition(),
                     prevVal,
-                    updateCntr);
+                    updateCntr,
+                    lsnrs != null);
             }
-            else if (dhtNodes.size() == 1) {
+            else if (lsnrs != null && dhtNodes.size() == 1) {
                 try {
-                    cctx.continuousQueries().onEntryUpdated(entry.key(), val, prevVal,
-                        entry.key().internal() || !cctx.userCache(), entry.partition(), true, false,
-                        updateCntr, updateReq.topologyVersion());
+                    cctx.continuousQueries().onEntryUpdated(
+                        lsnrs,
+                        entry.key(),
+                        val,
+                        prevVal,
+                        entry.key().internal() || !cctx.userCache(),
+                        entry.partition(),
+                        true,
+                        false,
+                        updateCntr,
+                        updateReq.topologyVersion());
                 }
                 catch (IgniteCheckedException e) {
                     U.warn(log, "Failed to send continuous query message. [key=" + entry.key() + ", newVal="
@@ -352,7 +374,7 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
             cctx.mvcc().removeAtomicFuture(version());
 
             if (err != null) {
-                if (!mappings.isEmpty()) {
+                if (!mappings.isEmpty() && lsnrs != null) {
                     Collection<KeyCacheObject> hndKeys = new ArrayList<>(keys.size());
 
                     exit: for (GridDhtAtomicUpdateRequest req : mappings.values()) {
@@ -362,7 +384,11 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
                             if (!hndKeys.contains(key)) {
                                 updateRes.addFailedKey(key, err);
 
-                                cctx.continuousQueries().skipUpdateEvent(key, req.partitionId(i), req.updateCounter(i),
+                                cctx.continuousQueries().skipUpdateEvent(
+                                    lsnrs,
+                                    key,
+                                    req.partitionId(i),
+                                    req.updateCounter(i),
                                     updateReq.topologyVersion());
 
                                 hndKeys.add(key);
@@ -378,27 +404,38 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
                         updateRes.addFailedKey(key, err);
             }
             else {
-                Collection<KeyCacheObject> hndKeys = new ArrayList<>(keys.size());
+                if (lsnrs != null) {
+                    Collection<KeyCacheObject> hndKeys = new ArrayList<>(keys.size());
 
-                exit: for (GridDhtAtomicUpdateRequest req : mappings.values()) {
-                    for (int i = 0; i < req.size(); i++) {
-                        KeyCacheObject key = req.key(i);
+                    exit: for (GridDhtAtomicUpdateRequest req : mappings.values()) {
+                        for (int i = 0; i < req.size(); i++) {
+                            KeyCacheObject key = req.key(i);
 
-                        if (!hndKeys.contains(key)) {
-                            try {
-                                cctx.continuousQueries().onEntryUpdated(key, req.value(i), req.localPreviousValue(i),
-                                    key.internal() || !cctx.userCache(), req.partitionId(i), true, false,
-                                    req.updateCounter(i), updateReq.topologyVersion());
-                            }
-                            catch (IgniteCheckedException e) {
-                                U.warn(log, "Failed to send continuous query message. [key=" + key + ", newVal="
-                                    + req.value(i) + ", err=" + e + "]");
-                            }
+                            if (!hndKeys.contains(key)) {
+                                try {
+                                    cctx.continuousQueries().onEntryUpdated(
+                                        lsnrs,
+                                        key,
+                                        req.value(i),
+                                        req.localPreviousValue(i),
+                                        key.internal() || !cctx.userCache(),
+                                        req.partitionId(i),
+                                        true,
+                                        false,
+                                        req.updateCounter(i),
+                                        updateReq.topologyVersion());
+                                }
+                                catch (IgniteCheckedException e) {
+                                    U.warn(log, "Failed to send continuous query message. [key=" + key +
+                                        ", newVal=" + req.value(i) +
+                                        ", err=" + e + "]");
+                                }
 
-                            hndKeys.add(key);
+                                hndKeys.add(key);
 
-                            if (hndKeys.size() == keys.size())
-                                break exit;
+                                if (hndKeys.size() == keys.size())
+                                    break exit;
+                            }
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index 7cc276f..e417cdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -49,6 +50,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Lite dht cache backup update request.
  */
+@IgniteCodeGeneratingFail // Need add 'cleanup' call in 'writeTo' method.
 public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements GridCacheDeployable {
     /** */
     private static final long serialVersionUID = 0L;
@@ -215,7 +217,6 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
 
         keys = new ArrayList<>();
         partIds = new ArrayList<>();
-        locPrevVals = new ArrayList<>();
 
         if (forceTransformBackups) {
             entryProcessors = new ArrayList<>();
@@ -240,7 +241,10 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
      * @param conflictExpireTime Conflict expire time (optional).
      * @param conflictVer Conflict version (optional).
      * @param addPrevVal If {@code true} adds previous value.
+     * @param partId Partition.
      * @param prevVal Previous value.
+     * @param updateCntr Update counter.
+     * @param storeLocPrevVal If {@code true} stores previous value.
      */
     public void addWriteValue(KeyCacheObject key,
         @Nullable CacheObject val,
@@ -251,12 +255,18 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
         boolean addPrevVal,
         int partId,
         @Nullable CacheObject prevVal,
-        @Nullable Long updateIdx) {
+        @Nullable Long updateCntr,
+        boolean storeLocPrevVal) {
         keys.add(key);
 
         partIds.add(partId);
 
-        locPrevVals.add(prevVal);
+        if (storeLocPrevVal) {
+            if (locPrevVals == null)
+                locPrevVals = new ArrayList<>();
+
+            locPrevVals.add(prevVal);
+        }
 
         if (forceTransformBackups) {
             assert entryProcessor != null;
@@ -273,11 +283,11 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
             prevVals.add(prevVal);
         }
 
-        if (updateIdx != null) {
+        if (updateCntr != null) {
             if (updateCntrs == null)
                 updateCntrs = new GridLongList();
 
-            updateCntrs.add(updateIdx);
+            updateCntrs.add(updateCntr);
         }
 
         // In case there is no conflict, do not create the list.
@@ -521,6 +531,8 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
      * @return Value.
      */
     @Nullable public CacheObject localPreviousValue(int idx) {
+        assert locPrevVals != null;
+
         return locPrevVals.get(idx);
     }
 
@@ -849,6 +861,8 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
 
         }
 
+        cleanup();
+
         return true;
     }
 
@@ -1048,6 +1062,20 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
         return reader.afterMessageRead(GridDhtAtomicUpdateRequest.class);
     }
 
+    /**
+     * Cleanup values not needed after message was sent.
+     */
+    private void cleanup() {
+        nearVals = null;
+        prevVals = null;
+
+        // Do not keep values if they are not needed for continuous query notification.
+        if (locPrevVals == null) {
+           vals = null;
+           locPrevVals = null;
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public byte directType() {
         return 38;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 8f0cab7..0d8f795 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -1107,7 +1107,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 next = null;
 
                 while (it.hasNext()) {
-                    final LazySwapEntry e = new LazySwapEntry(it.next(), keepBinary);
+                    final LazySwapEntry e = new LazySwapEntry(it.next());
 
                     if (filter != null) {
                         K key = (K)cctx.unwrapBinaryIfNeeded(e.key(), keepBinary);
@@ -2524,15 +2524,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         /** */
         private final Map.Entry<byte[], byte[]> e;
 
-        /** */
-        private boolean keepBinary;
-
         /**
          * @param e Entry with
          */
-        LazySwapEntry(Map.Entry<byte[], byte[]> e, boolean keepBinary) {
+        LazySwapEntry(Map.Entry<byte[], byte[]> e) {
             this.e = e;
-            this.keepBinary = keepBinary;
         }
 
         /** {@inheritDoc} */
@@ -2545,9 +2541,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         @Override protected V unmarshalValue() throws IgniteCheckedException {
             IgniteBiTuple<byte[], Byte> t = GridCacheSwapEntryImpl.getValue(e.getValue());
 
-            CacheObject obj = cctx.cacheObjects().toCacheObject(cctx.cacheObjectContext(), t.get2(), t.get1());
-
-            return (V)cctx.cacheObjectContext().unwrapBinaryIfNeeded(obj, keepBinary);
+            return (V)cctx.cacheObjects().toCacheObject(cctx.cacheObjectContext(), t.get2(), t.get1());
         }
 
         /** {@inheritDoc} */
@@ -2597,13 +2591,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         @Override protected V unmarshalValue() throws IgniteCheckedException {
             long ptr = GridCacheOffheapSwapEntry.valueAddress(valPtr.get1(), valPtr.get2());
 
-            CacheObject obj = cctx.fromOffheap(ptr, false);
-
-            V val = CU.value(obj, cctx, false);
-
-            assert val != null;
-
-            return val;
+            return (V)cctx.fromOffheap(ptr, false);
         }
 
         /** {@inheritDoc} */
@@ -2661,7 +2649,15 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             if (!filter.apply(key, val))
                 return null;
 
-            return new IgniteBiTuple<>(e.key(), (V)cctx.unwrapTemporary(e.value()));
+            if (key instanceof CacheObject)
+                ((CacheObject)key).prepareMarshal(cctx.cacheObjectContext());
+
+            val = (V)cctx.unwrapTemporary(e.value());
+
+            if (val instanceof CacheObject)
+                ((CacheObject)val).prepareMarshal(cctx.cacheObjectContext());
+
+            return new IgniteBiTuple<>(e.key(), val);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index 7e66ad3..cf9b439 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -882,8 +882,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
          * @return Continuous query entry.
          */
         private CacheContinuousQueryEntry skipEntry(CacheContinuousQueryEntry e) {
-            if (lastFiredCntr.get() > e.updateCounter() || e.updateCounter() == 1) {
-
+            if (lastFiredCntr.get() > e.updateCounter() || e.updateCounter() == 1L) {
                 e.markFiltered();
 
                 return e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
index 86abbef..dce04de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 /**
  * Continuous query listener.
  */
-interface CacheContinuousQueryListener<K, V> {
+public interface CacheContinuousQueryListener<K, V> {
     /**
      * Query execution callback.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 0e4cb40..cc59989 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.resources.LoggerResource;
+import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
 import static javax.cache.event.EventType.CREATED;
@@ -155,37 +156,102 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param lsnrs Listeners to notify.
+     * @param key Entry key.
      * @param partId Partition id.
      * @param updCntr Updated counter.
      * @param topVer Topology version.
      */
-    public void skipUpdateEvent(KeyCacheObject key, int partId, long updCntr, AffinityTopologyVersion topVer) {
-        if (lsnrCnt.get() > 0) {
-            for (CacheContinuousQueryListener lsnr : lsnrs.values()) {
-                CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
-                    cctx.cacheId(),
-                    UPDATED,
-                    key,
-                    null,
-                    null,
-                    lsnr.keepBinary(),
-                    partId,
-                    updCntr,
-                    topVer);
+    public void skipUpdateEvent(Map<UUID, CacheContinuousQueryListener> lsnrs,
+        KeyCacheObject key, int partId, long updCntr, AffinityTopologyVersion topVer) {
+        assert lsnrs != null;
 
-                CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
-                    cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
+        for (CacheContinuousQueryListener lsnr : lsnrs.values()) {
+            CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
+                cctx.cacheId(),
+                UPDATED,
+                key,
+                null,
+                null,
+                lsnr.keepBinary(),
+                partId,
+                updCntr,
+                topVer);
 
-                lsnr.skipUpdateEvent(evt, topVer);
-            }
+            CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
+                cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
+
+            lsnr.skipUpdateEvent(evt, topVer);
         }
     }
 
     /**
+     * @param internal Internal entry flag (internal key or not user cache).
+     * @param preload Whether update happened during preloading.
+     * @return Registered listeners.
+     */
+    @Nullable public Map<UUID, CacheContinuousQueryListener> updateListeners(
+        boolean internal,
+        boolean preload) {
+        if (preload && !internal)
+            return null;
+
+        ConcurrentMap<UUID, CacheContinuousQueryListener> lsnrCol;
+
+        if (internal)
+            lsnrCol = intLsnrCnt.get() > 0 ? intLsnrs : null;
+        else
+            lsnrCol = lsnrCnt.get() > 0 ? lsnrs : null;
+
+        return F.isEmpty(lsnrCol) ? null : lsnrCol;
+    }
+
+    /**
+     * @param key Key.
+     * @param newVal New value.
+     * @param oldVal Old value.
+     * @param internal Internal entry (internal key or not user cache).
+     * @param partId Partition.
+     * @param primary {@code True} if called on primary node.
+     * @param preload Whether update happened during preloading.
+     * @param updateCntr Update counter.
+     * @param topVer Topology version.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public void onEntryUpdated(
+        KeyCacheObject key,
+        CacheObject newVal,
+        CacheObject oldVal,
+        boolean internal,
+        int partId,
+        boolean primary,
+        boolean preload,
+        long updateCntr,
+        AffinityTopologyVersion topVer) throws IgniteCheckedException {
+        Map<UUID, CacheContinuousQueryListener> lsnrCol = updateListeners(internal, preload);
+
+        if (lsnrCol != null) {
+            onEntryUpdated(
+                lsnrCol,
+                key,
+                newVal,
+                oldVal,
+                internal,
+                partId,
+                primary,
+                preload,
+                updateCntr,
+                topVer);
+        }
+    }
+
+    /**
+     * @param lsnrCol Listeners to notify.
      * @param key Key.
      * @param newVal New value.
      * @param oldVal Old value.
      * @param internal Internal entry (internal key or not user cache),
+     * @param partId Partition.
      * @param primary {@code True} if called on primary node.
      * @param preload Whether update happened during preloading.
      * @param updateCntr Update counter.
@@ -193,6 +259,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @throws IgniteCheckedException In case of error.
      */
     public void onEntryUpdated(
+        Map<UUID, CacheContinuousQueryListener> lsnrCol,
         KeyCacheObject key,
         CacheObject newVal,
         CacheObject oldVal,
@@ -205,25 +272,16 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         throws IgniteCheckedException
     {
         assert key != null;
-
-        if (preload && !internal)
-            return;
-
-        ConcurrentMap<UUID, CacheContinuousQueryListener> lsnrCol;
-
-        if (internal)
-            lsnrCol = intLsnrCnt.get() > 0 ? intLsnrs : null;
-        else
-            lsnrCol = lsnrCnt.get() > 0 ? lsnrs : null;
-
-        if (F.isEmpty(lsnrCol))
-            return;
+        assert lsnrCol != null;
 
         boolean hasNewVal = newVal != null;
         boolean hasOldVal = oldVal != null;
 
-        if (!hasNewVal && !hasOldVal)
+        if (!hasNewVal && !hasOldVal) {
+            skipUpdateEvent(lsnrCol, key, partId, updateCntr, topVer);
+
             return;
+        }
 
         EventType evtType = !hasNewVal ? REMOVED : !hasOldVal ? CREATED : UPDATED;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 7c7e3e3..0218897 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -893,11 +894,18 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         }
 
         // Load partition counters.
-        if (hnd.isQuery() && ctx.cache() != null && ctx.cache().internalCache(hnd.cacheName()) != null) {
-            Map<Integer, Long> cntrs = ctx.cache().internalCache(hnd.cacheName())
-                .context().topology().updateCounters();
+        if (hnd.isQuery()) {
+            GridCacheProcessor proc = ctx.cache();
 
-            req.addUpdateCounters(cntrs);
+            if (proc != null) {
+                GridCacheAdapter cache = ctx.cache().internalCache(hnd.cacheName());
+
+                if (cache != null && !cache.isLocal()) {
+                    Map<Integer, Long> cntrs = cache.context().topology().updateCounters();
+
+                    req.addUpdateCounters(cntrs);
+                }
+            }
         }
 
         if (err != null)


[09/43] ignite git commit: IGNITE-2329: Implemented a bunch of optimizations: - Garbageless NIO Selector - Get rid of unnecessary ArrayList allocations in GridCacheMvccManager. - Optimized "force keys" futures logic.

Posted by vk...@apache.org.
IGNITE-2329: Implemented a bunch of optimizations:
- Garbageless NIO Selector
- Get rid of unnecessary ArrayList allocations in GridCacheMvccManager.
- Optimized "force keys" futures logic.


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

Branch: refs/heads/ignite-2249
Commit: 75961eee2513427d94a1c7e0dbb96ac46195544b
Parents: 4210989
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Feb 5 21:13:26 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Feb 5 21:13:26 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  12 +-
 .../processors/cache/GridCacheAdapter.java      |  37 +-
 .../processors/cache/GridCacheMvccManager.java  |  42 +-
 .../processors/cache/GridCachePreloader.java    |   6 +
 .../cache/GridCachePreloaderAdapter.java        |   5 +
 .../processors/cache/GridCacheUtils.java        |  21 +-
 .../dht/GridClientPartitionTopology.java        |   5 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  72 ++-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |  13 +-
 .../cache/distributed/dht/GridDhtGetFuture.java | 176 ++++---
 .../distributed/dht/GridDhtGetSingleFuture.java | 476 +++++++++++++++++++
 .../distributed/dht/GridDhtLocalPartition.java  |  76 +--
 .../distributed/dht/GridDhtPartitionState.java  |   2 +-
 .../dht/GridDhtPartitionTopology.java           |   5 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   9 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |   7 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  40 +-
 .../dht/preloader/GridDhtPreloader.java         |  16 +
 .../cache/distributed/near/GridNearTxLocal.java |   1 -
 .../IgniteCacheObjectProcessorImpl.java         |   2 +-
 .../util/future/GridCompoundFuture.java         |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 143 +++++-
 .../util/nio/GridSelectorNioSessionImpl.java    |   2 +-
 .../util/nio/SelectedSelectionKeySet.java       | 111 +++++
 .../org/apache/ignite/lang/IgniteBiTuple.java   |   6 +-
 .../IgniteTxPreloadAbstractTest.java            |   2 +-
 .../near/GridCacheNearReadersSelfTest.java      |  19 +-
 .../apache/ignite/lang/GridTupleSelfTest.java   |  42 +-
 parent/pom.xml                                  |   1 +
 30 files changed, 1119 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index de7c10b..6f07702 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -370,11 +370,21 @@ public final class IgniteSystemProperties {
     /**
      * Manages {@link OptimizedMarshaller} behavior of {@code serialVersionUID} computation for
      * {@link Serializable} classes.
-     * */
+     */
     public static final String IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID =
         "IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID";
 
     /**
+     * If set to {@code true}, then default selected keys set is used inside
+     * {@code GridNioServer} which lead to some extra garbage generation when
+     * processing selected keys.
+     * <p>
+     * Default value is {@code false}. Should be switched to {@code true} if there are
+     * any problems in communication layer.
+     */
+    public static final String IGNITE_NO_SELECTOR_OPTS = "IGNITE_NO_SELECTOR_OPTS";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 9f54ddb..84eb0b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -101,7 +101,6 @@ import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.GridLeanMap;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -1823,7 +1822,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param needVer If {@code true} returns values as tuples containing value and version.
      * @return Future.
      */
-    public final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(@Nullable final Collection<KeyCacheObject> keys,
+    public final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(
+        @Nullable final Collection<KeyCacheObject> keys,
         final boolean readThrough,
         boolean checkTx,
         @Nullable final UUID subjId,
@@ -1834,7 +1834,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         final boolean keepCacheObjects,
         boolean canRemap,
         final boolean needVer
-        ) {
+    ) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(Collections.<K1, V1>emptyMap());
 
@@ -1853,11 +1853,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         if (tx == null || tx.implicit()) {
             try {
-                final AffinityTopologyVersion topVer = tx == null
-                    ? (canRemap ? ctx.affinity().affinityTopologyVersion(): ctx.shared().exchange().readyAffinityVersion())
-                    : tx.topologyVersion();
+                final AffinityTopologyVersion topVer = tx == null ?
+                    (canRemap ?
+                        ctx.affinity().affinityTopologyVersion() : ctx.shared().exchange().readyAffinityVersion()) :
+                        tx.topologyVersion();
+
+                int keysSize = keys.size();
 
-                final Map<K1, V1> map = new GridLeanMap<>(keys.size());
+                final Map<K1, V1> map = keysSize == 1 ?
+                    (Map<K1, V1>)new IgniteBiTuple<>() :
+                    U.<K1, V1>newHashMap(keysSize);
 
                 final boolean storeEnabled = !skipVals && readThrough && ctx.readThrough();
 
@@ -1893,7 +1898,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                     GridCacheVersion ver = entry.version();
 
                                     if (misses == null)
-                                        misses = new GridLeanMap<>();
+                                        misses = new HashMap<>();
 
                                     misses.put(key, ver);
                                 }
@@ -1913,7 +1918,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                 if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED))
                                     ctx.evicts().touch(entry, topVer);
 
-                                if (keys.size() == 1)
+                                if (keysSize == 1)
                                     // Safe to return because no locks are required in READ_COMMITTED mode.
                                     return new GridFinishedFuture<>(map);
                             }
@@ -2051,17 +2056,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                         }
                     );
                 }
-                else {
-                    // If misses is not empty and store is disabled, we should touch missed entries.
-                    if (misses != null) {
-                        for (KeyCacheObject key : misses.keySet()) {
-                            GridCacheEntryEx entry = peekEx(key);
-
-                            if (entry != null)
-                                ctx.evicts().touch(entry, topVer);
-                        }
-                    }
-                }
+                else
+                    // Misses can be non-zero only if store is enabled.
+                    assert misses == null;
 
                 return new GridFinishedFuture<>(map);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index c7d1f62..b2c23f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -17,6 +17,18 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -52,18 +64,6 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.ConcurrentLinkedDeque8;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.util.GridConcurrentFactory.newMap;
@@ -77,12 +77,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     private static final int MAX_REMOVED_LOCKS = 10240;
 
     /** Pending locks per thread. */
-    private final ThreadLocal<LinkedList<GridCacheMvccCandidate>> pending =
-        new ThreadLocal<LinkedList<GridCacheMvccCandidate>>() {
-            @Override protected LinkedList<GridCacheMvccCandidate> initialValue() {
-                return new LinkedList<>();
-            }
-        };
+    private final ThreadLocal<Deque<GridCacheMvccCandidate>> pending = new ThreadLocal<>();
 
     /** Pending near local locks and topology version per thread. */
     private ConcurrentMap<Long, GridCacheExplicitLockSpan> pendingExplicit;
@@ -683,7 +678,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return Remote candidates.
      */
     public Collection<GridCacheMvccCandidate> remoteCandidates() {
-        Collection<GridCacheMvccCandidate> rmtCands = new LinkedList<>();
+        Collection<GridCacheMvccCandidate> rmtCands = new ArrayList<>();
 
         for (GridDistributedCacheEntry entry : locked())
             rmtCands.addAll(entry.remoteMvccSnapshot());
@@ -697,7 +692,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return Local candidates.
      */
     public Collection<GridCacheMvccCandidate> localCandidates() {
-        Collection<GridCacheMvccCandidate> locCands = new LinkedList<>();
+        Collection<GridCacheMvccCandidate> locCands = new ArrayList<>();
 
         for (GridDistributedCacheEntry entry : locked()) {
             try {
@@ -726,7 +721,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         if (cacheCtx.isNear() || cand.singleImplicit())
             return true;
 
-        LinkedList<GridCacheMvccCandidate> queue = pending.get();
+        Deque<GridCacheMvccCandidate> queue = pending.get();
+
+        if (queue == null)
+            pending.set(queue = new ArrayDeque<>());
 
         GridCacheMvccCandidate prev = null;
 
@@ -751,7 +749,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * Reset MVCC context.
      */
     public void contextReset() {
-        pending.set(new LinkedList<GridCacheMvccCandidate>());
+        pending.set(null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index c8fcb90..be019fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -137,6 +137,12 @@ public interface GridCachePreloader {
     public IgniteInternalFuture<Boolean> rebalanceFuture();
 
     /**
+     * @return {@code true} if there is no need to force keys preloading
+     *      (e.g. rebalancing has been completed).
+     */
+    public boolean needForceKeys();
+
+    /**
      * Requests that preloader sends the request for the key.
      *
      * @param keys Keys to request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index a1704fc..5d98c6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -93,6 +93,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean needForceKeys() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void onReconnected() {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/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 8723827..cd21794 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
@@ -751,23 +751,28 @@ public class GridCacheUtils {
      * @param <T> Collection element type.
      * @return Reducer.
      */
-    public static <T> IgniteReducer<Collection<T>, Collection<T>> collectionsReducer() {
+    public static <T> IgniteReducer<Collection<T>, Collection<T>> collectionsReducer(final int size) {
         return new IgniteReducer<Collection<T>, Collection<T>>() {
-            private final Collection<T> ret = new ConcurrentLinkedQueue<>();
+            private List<T> ret;
+
+            @Override public synchronized boolean collect(Collection<T> c) {
+                if (c == null)
+                    return true;
+
+                if (ret == null)
+                    ret = new ArrayList<>(size);
 
-            @Override public boolean collect(Collection<T> c) {
-                if (c != null)
-                    ret.addAll(c);
+                ret.addAll(c);
 
                 return true;
             }
 
-            @Override public Collection<T> reduce() {
-                return ret;
+            @Override public synchronized Collection<T> reduce() {
+                return ret == null ? Collections.<T>emptyList() : ret;
             }
 
             /** {@inheritDoc} */
-            @Override public String toString() {
+            @Override public synchronized String toString() {
                 return "Collection reducer: " + ret;
             }
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index dcfc038..ad4943e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -336,6 +336,11 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public void releasePartitions(int... parts) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public List<GridDhtLocalPartition> localPartitions() {
         return Collections.emptyList();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 5be4e72..8e456e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -698,7 +698,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         @Nullable UUID subjId,
         int taskNameHash,
         @Nullable IgniteCacheExpiryPolicy expiry,
-        boolean skipVals) {
+        boolean skipVals
+    ) {
         GridDhtGetFuture<K, V> fut = new GridDhtGetFuture<>(ctx,
             msgId,
             reader,
@@ -718,21 +719,63 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
     /**
      * @param nodeId Node ID.
+     * @param msgId Message ID.
+     * @param key Key.
+     * @param addRdr Add reader flag.
+     * @param readThrough Read through flag.
+     * @param topVer Topology version flag.
+     * @param subjId Subject ID.
+     * @param taskNameHash Task name hash.
+     * @param expiry Expiry.
+     * @param skipVals Skip vals flag.
+     * @return Future for the operation.
+     */
+    private IgniteInternalFuture<GridCacheEntryInfo> getDhtSingleAsync(
+        UUID nodeId,
+        long msgId,
+        KeyCacheObject key,
+        boolean addRdr,
+        boolean readThrough,
+        AffinityTopologyVersion topVer,
+        @Nullable UUID subjId,
+        int taskNameHash,
+        @Nullable IgniteCacheExpiryPolicy expiry,
+        boolean skipVals
+    ) {
+        GridDhtGetSingleFuture<K, V> fut = new GridDhtGetSingleFuture<>(
+            ctx,
+            msgId,
+            nodeId,
+            key,
+            addRdr,
+            readThrough,
+            /*tx*/null,
+            topVer,
+            subjId,
+            taskNameHash,
+            expiry,
+            skipVals);
+
+        fut.init();
+
+        return fut;
+    }
+
+    /**
+     * @param nodeId Node ID.
      * @param req Get request.
      */
     protected void processNearSingleGetRequest(final UUID nodeId, final GridNearSingleGetRequest req) {
         assert ctx.affinityNode();
 
-        long ttl = req.accessTtl();
-
-        final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(ttl);
-
-        Map<KeyCacheObject, Boolean> map = Collections.singletonMap(req.key(), req.addReader());
+        final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(req.accessTtl());
 
-        IgniteInternalFuture<Collection<GridCacheEntryInfo>> fut =
-            getDhtAsync(nodeId,
+        IgniteInternalFuture<GridCacheEntryInfo> fut =
+            getDhtSingleAsync(
+                nodeId,
                 req.messageId(),
-                map,
+                req.key(),
+                req.addReader(),
                 req.readThrough(),
                 req.topologyVersion(),
                 req.subjectId(),
@@ -740,19 +783,16 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                 expiryPlc,
                 req.skipValues());
 
-        fut.listen(new CI1<IgniteInternalFuture<Collection<GridCacheEntryInfo>>>() {
-            @Override public void apply(IgniteInternalFuture<Collection<GridCacheEntryInfo>> f) {
+        fut.listen(new CI1<IgniteInternalFuture<GridCacheEntryInfo>>() {
+            @Override public void apply(IgniteInternalFuture<GridCacheEntryInfo> f) {
                 GridNearSingleGetResponse res;
 
-                GridDhtFuture<Collection<GridCacheEntryInfo>> fut =
-                    (GridDhtFuture<Collection<GridCacheEntryInfo>>)f;
+                GridDhtFuture<GridCacheEntryInfo> fut = (GridDhtFuture<GridCacheEntryInfo>)f;
 
                 try {
-                    Collection<GridCacheEntryInfo> entries = fut.get();
+                    GridCacheEntryInfo info = fut.get();
 
                     if (F.isEmpty(fut.invalidPartitions())) {
-                        GridCacheEntryInfo info = F.first(entries);
-
                         Message res0 = null;
 
                         if (info != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
index 0d10a93..1b9f743 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.Collections;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteBiClosure;
 
@@ -32,10 +31,6 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Retries. */
-    @GridToStringInclude
-    private Collection<Integer> invalidParts;
-
     /**
      * @param c Closure.
      * @param embedded Embedded.
@@ -45,8 +40,6 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
         IgniteInternalFuture<B> embedded
     ) {
         super(c, embedded);
-
-        invalidParts = Collections.emptyList();
     }
 
     /**
@@ -58,17 +51,15 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
         IgniteBiClosure<B, Exception, IgniteInternalFuture<A>> c
     ) {
         super(embedded, c);
-
-        invalidParts = Collections.emptyList();
     }
 
     /** {@inheritDoc} */
     @Override public Collection<Integer> invalidPartitions() {
-        return invalidParts;
+        return Collections.emptyList();
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridDhtEmbeddedFuture.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index c926c13..fa753b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -82,7 +83,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private Map<KeyCacheObject, Boolean> keys;
 
     /** Reserved partitions. */
-    private Collection<GridDhtLocalPartition> parts = new HashSet<>();
+    private int[] parts;
 
     /** Future ID. */
     private IgniteUuid futId;
@@ -137,7 +138,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
         boolean skipVals
     ) {
-        super(CU.<GridCacheEntryInfo>collectionsReducer());
+        super(CU.<GridCacheEntryInfo>collectionsReducer(keys.size()));
 
         assert reader != null;
         assert !F.isEmpty(keys);
@@ -194,8 +195,8 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     @Override public boolean onDone(Collection<GridCacheEntryInfo> res, Throwable err) {
         if (super.onDone(res, err)) {
             // Release all partitions reserved by this future.
-            for (GridDhtLocalPartition part : parts)
-                part.release();
+            if (parts != null)
+                cctx.topology().releasePartitions(parts);
 
             return true;
         }
@@ -209,68 +210,92 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
     private void map(final Map<KeyCacheObject, Boolean> keys) {
         GridDhtFuture<Object> fut = cctx.dht().dhtPreloader().request(keys.keySet(), topVer);
 
-        if (!F.isEmpty(fut.invalidPartitions())) {
-            if (retries == null)
-                retries = new HashSet<>();
+        if (fut != null) {
+            if (!F.isEmpty(fut.invalidPartitions())) {
+                if (retries == null)
+                    retries = new HashSet<>();
 
-            retries.addAll(fut.invalidPartitions());
-        }
+                retries.addAll(fut.invalidPartitions());
+            }
 
-        add(new GridEmbeddedFuture<>(
-            new IgniteBiClosure<Object, Exception, Collection<GridCacheEntryInfo>>() {
-                @Override public Collection<GridCacheEntryInfo> apply(Object o, Exception e) {
-                    if (e != null) { // Check error first.
-                        if (log.isDebugEnabled())
-                            log.debug("Failed to request keys from preloader [keys=" + keys + ", err=" + e + ']');
+            add(new GridEmbeddedFuture<>(
+                new IgniteBiClosure<Object, Exception, Collection<GridCacheEntryInfo>>() {
+                    @Override public Collection<GridCacheEntryInfo> apply(Object o, Exception e) {
+                        if (e != null) { // Check error first.
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to request keys from preloader [keys=" + keys + ", err=" + e + ']');
 
-                        onDone(e);
+                            onDone(e);
+                        }
+                        else
+                            map0(keys);
+
+                        // Finish this one.
+                        return Collections.emptyList();
                     }
+                },
+                fut));
+        }
+        else
+            map0(keys);
+    }
 
-                    Map<KeyCacheObject, Boolean> mappedKeys = null;
+    /**
+     * @param keys Keys to map.
+     */
+    private void map0(Map<KeyCacheObject, Boolean> keys) {
+        Map<KeyCacheObject, Boolean> mappedKeys = null;
 
-                    // Assign keys to primary nodes.
-                    for (Map.Entry<KeyCacheObject, Boolean> key : keys.entrySet()) {
-                        int part = cctx.affinity().partition(key.getKey());
+        // Assign keys to primary nodes.
+        for (Map.Entry<KeyCacheObject, Boolean> key : keys.entrySet()) {
+            int part = cctx.affinity().partition(key.getKey());
 
-                        if (retries == null || !retries.contains(part)) {
-                            if (!map(key.getKey(), parts)) {
-                                if (retries == null)
-                                    retries = new HashSet<>();
+            if (retries == null || !retries.contains(part)) {
+                if (!map(key.getKey())) {
+                    if (retries == null)
+                        retries = new HashSet<>();
 
-                                retries.add(part);
+                    retries.add(part);
 
-                                if (mappedKeys == null) {
-                                    mappedKeys = U.newLinkedHashMap(keys.size());
+                    if (mappedKeys == null) {
+                        mappedKeys = U.newLinkedHashMap(keys.size());
 
-                                    for (Map.Entry<KeyCacheObject, Boolean> key1 : keys.entrySet()) {
-                                        if (key1.getKey() == key.getKey())
-                                            break;
+                        for (Map.Entry<KeyCacheObject, Boolean> key1 : keys.entrySet()) {
+                            if (key1.getKey() == key.getKey())
+                                break;
 
-                                        mappedKeys.put(key.getKey(), key1.getValue());
-                                    }
-                                }
-                            }
-                            else if (mappedKeys != null)
-                                mappedKeys.put(key.getKey(), key.getValue());
+                            mappedKeys.put(key.getKey(), key1.getValue());
                         }
                     }
+                }
+                else if (mappedKeys != null)
+                    mappedKeys.put(key.getKey(), key.getValue());
+            }
+        }
 
-                    // Add new future.
-                    add(getAsync(mappedKeys == null ? keys : mappedKeys));
+        // Add new future.
+        IgniteInternalFuture<Collection<GridCacheEntryInfo>> fut = getAsync(mappedKeys == null ? keys : mappedKeys);
 
-                    // Finish this one.
-                    return Collections.emptyList();
-                }
-            },
-            fut));
+        // Optimization to avoid going through compound future,
+        // if getAsync() has been completed and no other futures added to this
+        // compound future.
+        if (fut.isDone() && futuresSize() == 0) {
+            if (fut.error() != null)
+                onDone(fut.error());
+            else
+                onDone(fut.result());
+
+            return;
+        }
+
+        add(fut);
     }
 
     /**
      * @param key Key.
-     * @param parts Parts to map.
      * @return {@code True} if mapped.
      */
-    private boolean map(KeyCacheObject key, Collection<GridDhtLocalPartition> parts) {
+    private boolean map(KeyCacheObject key) {
         GridDhtLocalPartition part = topVer.topologyVersion() > 0 ?
             cache().topology().localPartition(cctx.affinity().partition(key), topVer, true) :
             cache().topology().localPartition(key, false);
@@ -278,10 +303,12 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
         if (part == null)
             return false;
 
-        if (!parts.contains(part)) {
+        if (parts == null || !F.contains(parts, part.id())) {
             // By reserving, we make sure that partition won't be unloaded while processed.
             if (part.reserve()) {
-                parts.add(part);
+                parts = parts == null ? new int[1] : Arrays.copyOf(parts, parts.length + 1);
+
+                parts[parts.length - 1] = part.id();
 
                 return true;
             }
@@ -422,37 +449,56 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
             );
         }
 
+        if (fut.isDone()) {
+            if (fut.error() != null)
+                onDone(fut.error());
+            else
+                return new GridFinishedFuture<>(toEntryInfos(fut.result()));
+        }
+
         return new GridEmbeddedFuture<>(
             new C2<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>, Exception, Collection<GridCacheEntryInfo>>() {
-                @Override public Collection<GridCacheEntryInfo> apply(Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>> map, Exception e) {
+                @Override public Collection<GridCacheEntryInfo> apply(
+                    Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>> map, Exception e
+                ) {
                     if (e != null) {
                         onDone(e);
 
                         return Collections.emptyList();
                     }
-                    else {
-                        Collection<GridCacheEntryInfo> infos = new ArrayList<>(map.size());
+                    else
+                        return toEntryInfos(map);
+                }
+            },
+            fut);
+    }
 
-                        for (Map.Entry<KeyCacheObject, T2<CacheObject, GridCacheVersion>> entry : map.entrySet()) {
-                            T2<CacheObject, GridCacheVersion> val = entry.getValue();
+    /**
+     * @param map Map to convert.
+     * @return List of infos.
+     */
+    private Collection<GridCacheEntryInfo> toEntryInfos(Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>> map) {
+        if (map.isEmpty())
+            return Collections.emptyList();
 
-                            assert val != null;
+        Collection<GridCacheEntryInfo> infos = new ArrayList<>(map.size());
 
-                            GridCacheEntryInfo info = new GridCacheEntryInfo();
+        for (Map.Entry<KeyCacheObject, T2<CacheObject, GridCacheVersion>> entry : map.entrySet()) {
+            T2<CacheObject, GridCacheVersion> val = entry.getValue();
 
-                            info.cacheId(cctx.cacheId());
-                            info.key(entry.getKey());
-                            info.value(skipVals ? null : val.get1());
-                            info.version(val.get2());
+            assert val != null;
 
-                            infos.add(info);
-                        }
+            GridCacheEntryInfo info = new GridCacheEntryInfo();
 
-                        return infos;
-                    }
-                }
-            },
-            fut);
+            info.cacheId(cctx.cacheId());
+            info.key(entry.getKey());
+            info.value(skipVals ? null : val.get1());
+            info.version(val.get2());
+
+            infos.add(info);
+        }
+
+        return infos;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
new file mode 100644
index 0000000..d9851c7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java
@@ -0,0 +1,476 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
+import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public final class GridDhtGetSingleFuture<K, V> extends GridFutureAdapter<GridCacheEntryInfo>
+    implements GridDhtFuture<GridCacheEntryInfo> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Logger reference. */
+    private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
+
+    /** Logger. */
+    private static IgniteLogger log;
+
+    /** Message ID. */
+    private long msgId;
+
+    /** */
+    private UUID reader;
+
+    /** Read through flag. */
+    private boolean readThrough;
+
+    /** Context. */
+    private GridCacheContext<K, V> cctx;
+
+    /** Key. */
+    private KeyCacheObject key;
+
+    /** */
+    private boolean addRdr;
+
+    /** Reserved partitions. */
+    private int part = -1;
+
+    /** Future ID. */
+    private IgniteUuid futId;
+
+    /** Version. */
+    private GridCacheVersion ver;
+
+    /** Topology version .*/
+    private AffinityTopologyVersion topVer;
+
+    /** Transaction. */
+    private IgniteTxLocalEx tx;
+
+    /** Retries because ownership changed. */
+    private Collection<Integer> retries;
+
+    /** Subject ID. */
+    private UUID subjId;
+
+    /** Task name. */
+    private int taskNameHash;
+
+    /** Expiry policy. */
+    private IgniteCacheExpiryPolicy expiryPlc;
+
+    /** Skip values flag. */
+    private boolean skipVals;
+
+    /**
+     * @param cctx Context.
+     * @param msgId Message ID.
+     * @param reader Reader.
+     * @param key Key.
+     * @param addRdr Add reader flag.
+     * @param readThrough Read through flag.
+     * @param tx Transaction.
+     * @param topVer Topology version.
+     * @param subjId Subject ID.
+     * @param taskNameHash Task name hash code.
+     * @param expiryPlc Expiry policy.
+     * @param skipVals Skip values flag.
+     */
+    public GridDhtGetSingleFuture(
+        GridCacheContext<K, V> cctx,
+        long msgId,
+        UUID reader,
+        KeyCacheObject key,
+        Boolean addRdr,
+        boolean readThrough,
+        @Nullable IgniteTxLocalEx tx,
+        @NotNull AffinityTopologyVersion topVer,
+        @Nullable UUID subjId,
+        int taskNameHash,
+        @Nullable IgniteCacheExpiryPolicy expiryPlc,
+        boolean skipVals
+    ) {
+        assert reader != null;
+        assert key != null;
+
+        this.reader = reader;
+        this.cctx = cctx;
+        this.msgId = msgId;
+        this.key = key;
+        this.addRdr = addRdr;
+        this.readThrough = readThrough;
+        this.tx = tx;
+        this.topVer = topVer;
+        this.subjId = subjId;
+        this.taskNameHash = taskNameHash;
+        this.expiryPlc = expiryPlc;
+        this.skipVals = skipVals;
+
+        futId = IgniteUuid.randomUuid();
+
+        ver = tx == null ? cctx.versions().next() : tx.xidVersion();
+
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridDhtGetSingleFuture.class);
+    }
+
+    /**
+     * Initializes future.
+     */
+    void init() {
+        map();
+    }
+
+    /**
+     * @return Future ID.
+     */
+    public IgniteUuid futureId() {
+        return futId;
+    }
+
+    /**
+     * @return Future version.
+     */
+    public GridCacheVersion version() {
+        return ver;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onDone(GridCacheEntryInfo res, Throwable err) {
+        if (super.onDone(res, err)) {
+            // Release all partitions reserved by this future.
+            if (part != -1)
+                cctx.topology().releasePartitions(part);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     *
+     */
+    private void map() {
+        if (cctx.dht().dhtPreloader().needForceKeys()) {
+            GridDhtFuture<Object> fut = cctx.dht().dhtPreloader().request(
+                Collections.singleton(key),
+                topVer);
+
+            if (fut != null) {
+                if (F.isEmpty(fut.invalidPartitions())) {
+                    if (retries == null)
+                        retries = new HashSet<>();
+
+                    retries.addAll(fut.invalidPartitions());
+                }
+
+                fut.listen(
+                    new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                        @Override public void apply(IgniteInternalFuture<Object> fut) {
+                            Throwable e = fut.error();
+
+                            if (e != null) { // Check error first.
+                                if (log.isDebugEnabled())
+                                    log.debug("Failed to request keys from preloader " +
+                                        "[keys=" + key + ", err=" + e + ']');
+
+                                onDone(e);
+                            }
+                            else
+                                map0();
+                        }
+                    }
+                );
+
+                return;
+            }
+        }
+
+        map0();
+    }
+
+    /**
+     *
+     */
+    private void map0() {
+        // Assign keys to primary nodes.
+        int part = cctx.affinity().partition(key);
+
+        if (retries == null || !retries.contains(part)) {
+            if (!map(key)) {
+                retries = Collections.singleton(part);
+
+                onDone((GridCacheEntryInfo)null);
+
+                return;
+            }
+        }
+
+        getAsync();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> invalidPartitions() {
+        return retries == null ? Collections.<Integer>emptyList() : retries;
+    }
+
+    /**
+     * @param key Key.
+     * @return {@code True} if mapped.
+     */
+    private boolean map(KeyCacheObject key) {
+        GridDhtLocalPartition part = topVer.topologyVersion() > 0 ?
+            cache().topology().localPartition(cctx.affinity().partition(key), topVer, true) :
+            cache().topology().localPartition(key, false);
+
+        if (part == null)
+            return false;
+
+        assert this.part == -1;
+
+        // By reserving, we make sure that partition won't be unloaded while processed.
+        if (part.reserve()) {
+            this.part = part.id();
+
+            return true;
+        }
+        else
+            return false;
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings( {"unchecked", "IfMayBeConditional"})
+    private void getAsync() {
+        assert part != -1;
+
+        String taskName0 = cctx.kernalContext().job().currentTaskName();
+
+        if (taskName0 == null)
+            taskName0 = cctx.kernalContext().task().resolveTaskName(taskNameHash);
+
+        final String taskName = taskName0;
+
+        IgniteInternalFuture<Boolean> rdrFut = null;
+
+        ClusterNode readerNode = cctx.discovery().node(reader);
+
+        if (readerNode != null && !readerNode.isLocal() && cctx.discovery().cacheNearNode(readerNode, cctx.name())) {
+            while (true) {
+                GridDhtCacheEntry e = cache().entryExx(key, topVer);
+
+                try {
+                    if (e.obsolete())
+                        continue;
+
+                    boolean addReader = (!e.deleted() && addRdr && !skipVals);
+
+                    if (addReader)
+                        e.unswap(false);
+
+                    // Register reader. If there are active transactions for this entry,
+                    // then will wait for their completion before proceeding.
+                    // TODO: GG-4003:
+                    // TODO: What if any transaction we wait for actually removes this entry?
+                    // TODO: In this case seems like we will be stuck with untracked near entry.
+                    // TODO: To fix, check that reader is contained in the list of readers once
+                    // TODO: again after the returned future completes - if not, try again.
+                    rdrFut = addReader ? e.addReader(reader, msgId, topVer) : null;
+
+                    break;
+                }
+                catch (IgniteCheckedException err) {
+                    onDone(err);
+
+                    return;
+                }
+                catch (GridCacheEntryRemovedException ignore) {
+                    if (log.isDebugEnabled())
+                        log.debug("Got removed entry when getting a DHT value: " + e);
+                }
+                finally {
+                    cctx.evicts().touch(e, topVer);
+                }
+            }
+        }
+
+        IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>> fut;
+
+        if (rdrFut == null || rdrFut.isDone()) {
+            if (tx == null) {
+                fut = cache().getDhtAllAsync(
+                    Collections.singleton(key),
+                    readThrough,
+                    subjId,
+                    taskName,
+                    expiryPlc,
+                    skipVals,
+                    /*can remap*/true);
+            }
+            else {
+                fut = tx.getAllAsync(cctx,
+                    Collections.singleton(key),
+                    /*deserialize binary*/false,
+                    skipVals,
+                    /*keep cache objects*/true,
+                    /*skip store*/!readThrough,
+                    false);
+            }
+        }
+        else {
+            rdrFut.listen(
+                new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+                    @Override public void apply(IgniteInternalFuture<Boolean> fut) {
+                        Throwable e = fut.error();
+
+                        if (e != null) {
+                            onDone(e);
+
+                            return;
+                        }
+
+                        IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>> fut0;
+
+                        if (tx == null) {
+                            fut0 = cache().getDhtAllAsync(
+                                Collections.singleton(key),
+                                readThrough,
+                                subjId,
+                                taskName,
+                                expiryPlc,
+                                skipVals,
+                                /*can remap*/true);
+                        }
+                        else {
+                            fut0 = tx.getAllAsync(cctx,
+                                Collections.singleton(key),
+                                /*deserialize binary*/false,
+                                skipVals,
+                                /*keep cache objects*/true,
+                                /*skip store*/!readThrough,
+                                false
+                            );
+                        }
+
+                        fut0.listen(createGetFutureListener());
+                    }
+                }
+            );
+
+            return;
+        }
+
+        if (fut.isDone())
+            onResult(fut);
+        else
+            fut.listen(createGetFutureListener());
+    }
+
+    /**
+     * @return Listener for get future.
+     */
+    @NotNull private IgniteInClosure<IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>>>
+    createGetFutureListener() {
+        return new IgniteInClosure<IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>>>() {
+            @Override public void apply(
+                IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>> fut
+            ) {
+                onResult(fut);
+            }
+        };
+    }
+
+    /**
+     * @param fut Completed future to finish this process with.
+     */
+    private void onResult(IgniteInternalFuture<Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>>> fut) {
+        assert fut.isDone();
+
+        if (fut.error() != null)
+            onDone(fut.error());
+        else {
+            try {
+                onDone(toEntryInfo(fut.get()));
+            }
+            catch (IgniteCheckedException e) {
+                assert false; // Should never happen.
+            }
+        }
+    }
+
+    /**
+     * @param map Map to convert.
+     * @return List of infos.
+     */
+    private GridCacheEntryInfo toEntryInfo(Map<KeyCacheObject, T2<CacheObject, GridCacheVersion>> map) {
+        if (map.isEmpty())
+            return null;
+
+        T2<CacheObject, GridCacheVersion> val = map.get(key);
+
+        assert val != null;
+
+        GridCacheEntryInfo info = new GridCacheEntryInfo();
+
+        info.cacheId(cctx.cacheId());
+        info.key(key);
+        info.value(skipVals ? null : val.get1());
+        info.version(val.get2());
+
+        return info;
+    }
+
+    /**
+     * @return DHT cache.
+     */
+    private GridDhtCacheAdapter<K, V> cache() {
+        return (GridDhtCacheAdapter<K, V>)cctx.cache();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/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 c4312b5..4fc1eaf 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
@@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.AtomicStampedReference;
 import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -83,8 +82,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
     /** State. */
     @GridToStringExclude
-    private final AtomicStampedReference<GridDhtPartitionState> state =
-        new AtomicStampedReference<>(MOVING, 0);
+    private final AtomicLong state = new AtomicLong((long)MOVING.ordinal() << 32);
 
     /** Rent future. */
     @GridToStringExclude
@@ -153,8 +151,9 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * @return {@code false} If such reservation already added.
      */
     public boolean addReservation(GridDhtPartitionsReservation r) {
-        assert state.getReference() != EVICTED : "we can reserve only active partitions";
-        assert state.getStamp() != 0 : "partition must be already reserved before adding group reservation";
+        assert GridDhtPartitionState.fromOrdinal((int)(state.get() >> 32)) != EVICTED :
+            "we can reserve only active partitions";
+        assert (state.get() & 0xFFFF) != 0 : "partition must be already reserved before adding group reservation";
 
         return reservations.addIfAbsent(r);
     }
@@ -185,14 +184,14 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * @return Partition state.
      */
     public GridDhtPartitionState state() {
-        return state.getReference();
+        return GridDhtPartitionState.fromOrdinal((int)(state.get() >> 32));
     }
 
     /**
      * @return Reservations.
      */
     public int reservations() {
-        return state.getStamp();
+        return (int)(state.get() & 0xFFFF);
     }
 
     /**
@@ -385,14 +384,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     @Override public boolean reserve() {
         while (true) {
-            int reservations = state.getStamp();
+            long reservations = state.get();
 
-            GridDhtPartitionState s = state.getReference();
-
-            if (s == EVICTED)
+            if ((int)(reservations >> 32) == EVICTED.ordinal())
                 return false;
 
-            if (state.compareAndSet(s, s, reservations, reservations + 1))
+            if (state.compareAndSet(reservations, reservations + 1))
                 return true;
         }
     }
@@ -402,17 +399,15 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     @Override public void release() {
         while (true) {
-            int reservations = state.getStamp();
+            long reservations = state.get();
 
-            if (reservations == 0)
+            if ((int)(reservations & 0xFFFF) == 0)
                 return;
 
-            GridDhtPartitionState s = state.getReference();
-
-            assert s != EVICTED;
+            assert (int)(reservations >> 32) != EVICTED.ordinal();
 
             // Decrement reservations.
-            if (state.compareAndSet(s, s, reservations, --reservations)) {
+            if (state.compareAndSet(reservations, --reservations)) {
                 tryEvict();
 
                 break;
@@ -421,23 +416,32 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
+     * @param reservations Current aggregated value.
+     * @param toState State to switch to.
+     * @return {@code true} if cas succeeds.
+     */
+    private boolean casState(long reservations, GridDhtPartitionState toState) {
+        return state.compareAndSet(reservations, (reservations & 0xFFFF) | ((long)toState.ordinal() << 32));
+    }
+
+    /**
      * @return {@code True} if transitioned to OWNING state.
      */
     boolean own() {
         while (true) {
-            int reservations = state.getStamp();
+            long reservations = state.get();
 
-            GridDhtPartitionState s = state.getReference();
+            int ord = (int)(reservations >> 32);
 
-            if (s == RENTING || s == EVICTED)
+            if (ord == RENTING.ordinal() || ord == EVICTED.ordinal())
                 return false;
 
-            if (s == OWNING)
+            if (ord == OWNING.ordinal())
                 return true;
 
-            assert s == MOVING;
+            assert ord == MOVING.ordinal();
 
-            if (state.compareAndSet(MOVING, OWNING, reservations, reservations)) {
+            if (casState(reservations, OWNING)) {
                 if (log.isDebugEnabled())
                     log.debug("Owned partition: " + this);
 
@@ -455,14 +459,14 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      */
     IgniteInternalFuture<?> rent(boolean updateSeq) {
         while (true) {
-            int reservations = state.getStamp();
+            long reservations = state.get();
 
-            GridDhtPartitionState s = state.getReference();
+            int ord = (int)(reservations >> 32);
 
-            if (s == RENTING || s == EVICTED)
+            if (ord == RENTING.ordinal() || ord == EVICTED.ordinal())
                 return rent;
 
-            if (state.compareAndSet(s, RENTING, reservations, reservations)) {
+            if (casState(reservations, RENTING)) {
                 if (log.isDebugEnabled())
                     log.debug("Moved partition to RENTING state: " + this);
 
@@ -481,9 +485,13 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * @param updateSeq Update sequence.
      */
     void tryEvictAsync(boolean updateSeq) {
+        long reservations = state.get();
+
+        int ord = (int)(reservations >> 32);
+
         if (map.isEmpty() && !GridQueryProcessor.isEnabled(cctx.config()) &&
-            state.getReference() == RENTING && state.getStamp() == 0 &&
-            state.compareAndSet(RENTING, EVICTED, 0, 0)) {
+            ord == RENTING.ordinal() && (reservations & 0xFFFF) == 0 &&
+            casState(reservations, EVICTED)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);
 
@@ -520,13 +528,17 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      *
      */
     public void tryEvict() {
-        if (state.getReference() != RENTING || state.getStamp() != 0 || groupReserved())
+        long reservations = state.get();
+
+        int ord = (int)(reservations >> 32);
+
+        if (ord != RENTING.ordinal() || (reservations & 0xFFFF) != 0 || groupReserved())
             return;
 
         // Attempt to evict partition entries from cache.
         clearAll();
 
-        if (map.isEmpty() && state.compareAndSet(RENTING, EVICTED, 0, 0)) {
+        if (map.isEmpty() && casState(reservations, EVICTED)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionState.java
index 7b49369..041f135 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionState.java
@@ -52,4 +52,4 @@ public enum GridDhtPartitionState {
     public boolean active() {
         return this != EVICTED;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index dd06d6f..84889f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -112,6 +112,11 @@ public interface GridDhtPartitionTopology {
         throws GridDhtInvalidPartitionException;
 
     /**
+     * @param parts Partitions to release (should be reserved before).
+     */
+    public void releasePartitions(int... parts);
+
+    /**
      * @param key Cache key.
      * @param create If {@code true}, then partition will be created if it's not there.
      * @return Local partition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index d6fc8f1..0e579ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -612,6 +612,15 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public void releasePartitions(int... parts) {
+        assert parts != null;
+        assert parts.length > 0;
+
+        for (int i = 0; i < parts.length; i++)
+            locParts.get(parts[i]).release();
+    }
+
+    /** {@inheritDoc} */
     @Override public GridDhtLocalPartition localPartition(Object key, boolean create) {
         return localPartition(cctx.affinity().partition(key), AffinityTopologyVersion.NONE, create);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 41b28d5..4c783f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -988,7 +988,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
             lastForceFut = cctx.cacheContext(cacheId).preloader().request(keys, tx.topologyVersion());
 
-            if (compFut != null)
+            if (compFut != null && lastForceFut != null)
                 compFut.add(lastForceFut);
         }
 
@@ -997,11 +997,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
             return compFut;
         }
-        else {
-            assert lastForceFut != null;
-
+        else
             return lastForceFut;
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index f6f57ee..6c7bac5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1309,7 +1309,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     ) {
         IgniteInternalFuture<Object> forceFut = preldr.request(req.keys(), req.topologyVersion());
 
-        if (forceFut.isDone())
+        if (forceFut == null || forceFut.isDone())
             updateAllAsyncInternal0(nodeId, req, completionCb);
         else {
             forceFut.listen(new CI1<IgniteInternalFuture<Object>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index dc4b6bd..1a2eb22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -897,28 +897,24 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         IgniteInternalFuture<Object> keyFut = ctx.dht().dhtPreloader().request(keys, topVer);
 
         // Prevent embedded future creation if possible.
-        if (keyFut.isDone()) {
-            try {
-                // Check for exception.
-                keyFut.get();
-
-                return lockAllAsync0(cacheCtx,
-                    tx,
-                    threadId,
-                    ver,
-                    topVer,
-                    keys,
-                    txRead,
-                    retval,
-                    timeout,
-                    accessTtl,
-                    filter,
-                    skipStore,
-                    keepBinary);
-            }
-            catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(e);
-            }
+        if (keyFut == null || keyFut.isDone()) {
+            // Check for exception.
+            if (keyFut != null && keyFut.error() != null)
+                return new GridFinishedFuture<>(keyFut.error());
+
+            return lockAllAsync0(cacheCtx,
+                tx,
+                threadId,
+                ver,
+                topVer,
+                keys,
+                txRead,
+                retval,
+                timeout,
+                accessTtl,
+                filter,
+                skipStore,
+                keepBinary);
         }
         else {
             return new GridEmbeddedFuture<>(keyFut,

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index f0054e4..6ec02a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -403,6 +403,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
         try {
             demandLock.readLock().lock();
+
             try {
                 demander.handleSupplyMessage(idx, id, s);
             }
@@ -692,12 +693,27 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean needForceKeys() {
+        if (cctx.rebalanceEnabled()) {
+            IgniteInternalFuture<Boolean> rebalanceFut = rebalanceFuture();
+
+            if (rebalanceFut.isDone() && Boolean.TRUE.equals(rebalanceFut.result()))
+                return false;
+        }
+
+        return true;
+    }
+
     /**
      * @param keys Keys to request.
      * @return Future for request.
      */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
     @Override public GridDhtFuture<Object> request(Collection<KeyCacheObject> keys, AffinityTopologyVersion topVer) {
+        if (!needForceKeys())
+            return null;
+
         final GridDhtForceKeysFuture<?, ?> fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this);
 
         IgniteInternalFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index b7b480e..0853b77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 54dd69e..2e825b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -302,7 +302,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                     ClassLoader ldr = ctx.p2pEnabled() ?
                         IgniteUtils.detectClassLoader(IgniteUtils.detectClass(this.val)) : U.gridClassLoader();
 
-                     Object val = ctx.processor().unmarshal(ctx, valBytes, ldr);
+                    Object val = ctx.processor().unmarshal(ctx, valBytes, ldr);
 
                     return new KeyCacheObjectImpl(val, valBytes);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index c382497..3409341 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -258,7 +258,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> implements Ig
     /**
      * @return Futures size.
      */
-    private int futuresSize() {
+    protected int futuresSize() {
         synchronized (futs) {
             return futs.size();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c7679c0..75fa9f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.nio;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
@@ -43,10 +44,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -102,6 +103,24 @@ public class GridNioServer<T> {
     /** SSL write buf limit. */
     private static final int WRITE_BUF_LIMIT = GridNioSessionMetaKey.nextUniqueKey();
 
+    /** */
+    private static final boolean DISABLE_KEYSET_OPTIMIZATION =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_NO_SELECTOR_OPTS);
+
+    /**
+     *
+     */
+    static {
+        // This is a workaround for JDK bug (NPE in Selector.open()).
+        // http://bugs.sun.com/view_bug.do?bug_id=6427854
+        try {
+            Selector.open().close();
+        }
+        catch (IOException ignored) {
+            // No-op.
+        }
+    }
+
     /** Accept worker thread. */
     @GridToStringExclude
     private final IgniteThread acceptThread;
@@ -184,17 +203,6 @@ public class GridNioServer<T> {
     /** Optional listener to monitor outbound message queue size. */
     private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
-    /** Static initializer ensures single-threaded execution of workaround. */
-    static {
-        // This is a workaround for JDK bug (NPE in Selector.open()).
-        // http://bugs.sun.com/view_bug.do?bug_id=6427854
-        try {
-            Selector.open().close();
-        }
-        catch (IOException ignored) {
-        }
-    }
-
     /**
      * @param addr Address.
      * @param port Port.
@@ -445,10 +453,8 @@ public class GridNioServer<T> {
             // Change from 0 to 1 means that worker thread should be waken up.
             clientWorkers.get(ses.selectorIndex()).offer(fut);
 
-        IgniteBiInClosure<GridNioSession, Integer> lsnr0 = msgQueueLsnr;
-
-        if (lsnr0 != null)
-            lsnr0.apply(ses, msgCnt);
+        if (msgQueueLsnr != null)
+            msgQueueLsnr.apply(ses, msgCnt);
     }
 
     /**
@@ -1239,6 +1245,9 @@ public class GridNioServer<T> {
         /** Selector to select read events. */
         private Selector selector;
 
+        /** Selected keys. */
+        private SelectedSelectionKeySet selectedKeys;
+
         /** Worker index. */
         private final int idx;
 
@@ -1253,7 +1262,7 @@ public class GridNioServer<T> {
             throws IgniteCheckedException {
             super(gridName, name, log);
 
-            selector = createSelector(null);
+            createSelector();
 
             this.idx = idx;
         }
@@ -1262,10 +1271,11 @@ public class GridNioServer<T> {
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             try {
                 boolean reset = false;
+
                 while (!closed) {
                     try {
                         if (reset)
-                            selector = createSelector(null);
+                            createSelector();
 
                         bodyInternal();
                     }
@@ -1290,6 +1300,50 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @throws IgniteCheckedException If failed.
+         */
+        private void createSelector() throws IgniteCheckedException {
+            selectedKeys = null;
+
+            selector = GridNioServer.this.createSelector(null);
+
+            if (DISABLE_KEYSET_OPTIMIZATION)
+                return;
+
+            try {
+                SelectedSelectionKeySet selectedKeySet = new SelectedSelectionKeySet();
+
+                Class<?> selectorImplClass =
+                    Class.forName("sun.nio.ch.SelectorImpl", false, U.gridClassLoader());
+
+                // Ensure the current selector implementation is what we can instrument.
+                if (!selectorImplClass.isAssignableFrom(selector.getClass()))
+                    return;
+
+                Field selectedKeysField = selectorImplClass.getDeclaredField("selectedKeys");
+                Field publicSelectedKeysField = selectorImplClass.getDeclaredField("publicSelectedKeys");
+
+                selectedKeysField.setAccessible(true);
+                publicSelectedKeysField.setAccessible(true);
+
+                selectedKeysField.set(selector, selectedKeySet);
+                publicSelectedKeysField.set(selector, selectedKeySet);
+
+                selectedKeys = selectedKeySet;
+
+                if (log.isDebugEnabled())
+                    log.debug("Instrumented an optimized java.util.Set into: " + selector);
+            }
+            catch (Exception e) {
+                selectedKeys = null;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to instrument an optimized java.util.Set into selector [selector=" + selector
+                        + ", err=" + e + ']');
+            }
+        }
+
+        /**
          * Adds socket channel to the registration queue and wakes up reading thread.
          *
          * @param req Change request.
@@ -1385,7 +1439,10 @@ public class GridNioServer<T> {
                     // Wake up every 2 seconds to check if closed.
                     if (selector.select(2000) > 0) {
                         // Walk through the ready keys collection and process network events.
-                        processSelectedKeys(selector.selectedKeys());
+                        if (selectedKeys == null)
+                            processSelectedKeys(selector.selectedKeys());
+                        else
+                            processSelectedKeysOptimized(selectedKeys.flip());
                     }
 
                     long now = U.currentTimeMillis();
@@ -1431,10 +1488,58 @@ public class GridNioServer<T> {
          * @param keys Selected keys.
          * @throws ClosedByInterruptException If this thread was interrupted while reading data.
          */
+        private void processSelectedKeysOptimized(SelectionKey[] keys) throws ClosedByInterruptException {
+            for (int i = 0; ; i ++) {
+                final SelectionKey key = keys[i];
+
+                if (key == null)
+                    break;
+
+                // null out entry in the array to allow to have it GC'ed once the Channel close
+                // See https://github.com/netty/netty/issues/2363
+                keys[i] = null;
+
+                // Was key closed?
+                if (!key.isValid())
+                    continue;
+
+                GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+                assert ses != null;
+
+                try {
+                    if (key.isReadable())
+                        processRead(key);
+
+                    if (key.isValid() && key.isWritable())
+                        processWrite(key);
+                }
+                catch (ClosedByInterruptException e) {
+                    // This exception will be handled in bodyInternal() method.
+                    throw e;
+                }
+                catch (Exception e) {
+                    if (!closed)
+                        U.warn(log, "Failed to process selector key (will close): " + ses, e);
+
+                    close(ses, new GridNioException(e));
+                }
+            }
+        }
+
+        /**
+         * Processes keys selected by a selector.
+         *
+         * @param keys Selected keys.
+         * @throws ClosedByInterruptException If this thread was interrupted while reading data.
+         */
         private void processSelectedKeys(Set<SelectionKey> keys) throws ClosedByInterruptException {
             if (log.isTraceEnabled())
                 log.trace("Processing keys in client worker: " + keys.size());
 
+            if (keys.isEmpty())
+                return;
+
             for (Iterator<SelectionKey> iter = keys.iterator(); iter.hasNext(); ) {
                 SelectionKey key = iter.next();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index deb7d2b..1241f99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -309,4 +309,4 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     @Override public String toString() {
         return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
new file mode 100644
index 0000000..9aa245d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2013 The Netty Project
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+
+import java.nio.channels.SelectionKey;
+import java.util.AbstractSet;
+import java.util.Iterator;
+
+final class SelectedSelectionKeySet extends AbstractSet<SelectionKey> {
+
+    private SelectionKey[] keysA;
+    private int keysASize;
+    private SelectionKey[] keysB;
+    private int keysBSize;
+    private boolean isA = true;
+
+    SelectedSelectionKeySet() {
+        keysA = new SelectionKey[1024];
+        keysB = keysA.clone();
+    }
+
+    @Override
+    public boolean add(SelectionKey o) {
+        if (o == null) {
+            return false;
+        }
+
+        if (isA) {
+            int size = keysASize;
+            keysA[size ++] = o;
+            keysASize = size;
+            if (size == keysA.length) {
+                doubleCapacityA();
+            }
+        } else {
+            int size = keysBSize;
+            keysB[size ++] = o;
+            keysBSize = size;
+            if (size == keysB.length) {
+                doubleCapacityB();
+            }
+        }
+
+        return true;
+    }
+
+    private void doubleCapacityA() {
+        SelectionKey[] newKeysA = new SelectionKey[keysA.length << 1];
+        System.arraycopy(keysA, 0, newKeysA, 0, keysASize);
+        keysA = newKeysA;
+    }
+
+    private void doubleCapacityB() {
+        SelectionKey[] newKeysB = new SelectionKey[keysB.length << 1];
+        System.arraycopy(keysB, 0, newKeysB, 0, keysBSize);
+        keysB = newKeysB;
+    }
+
+    SelectionKey[] flip() {
+        if (isA) {
+            isA = false;
+            keysA[keysASize] = null;
+            keysBSize = 0;
+            return keysA;
+        } else {
+            isA = true;
+            keysB[keysBSize] = null;
+            keysASize = 0;
+            return keysB;
+        }
+    }
+
+    @Override
+    public int size() {
+        if (isA) {
+            return keysASize;
+        } else {
+            return keysBSize;
+        }
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        return false;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return false;
+    }
+
+    @Override
+    public Iterator<SelectionKey> iterator() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/main/java/org/apache/ignite/lang/IgniteBiTuple.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteBiTuple.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteBiTuple.java
index 6098007..89e5f16 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteBiTuple.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteBiTuple.java
@@ -250,7 +250,9 @@ public class IgniteBiTuple<V1, V2> implements Map<V1, V2>, Map.Entry<V1, V2>,
 
     /** {@inheritDoc} */
     @Override public Set<Map.Entry<V1, V2>> entrySet() {
-        return Collections.<Entry<V1, V2>>singleton(this);
+        return isEmpty() ?
+            Collections.<Entry<V1,V2>>emptySet() :
+            Collections.<Entry<V1, V2>>singleton(this);
     }
 
     /** {@inheritDoc} */
@@ -301,4 +303,4 @@ public class IgniteBiTuple<V1, V2> implements Map<V1, V2>, Map.Entry<V1, V2>,
     @Override public String toString() {
         return S.toString(IgniteBiTuple.class, this);
     }
-}
\ No newline at end of file
+}


[23/43] ignite git commit: IGNITE-2380: Added ability to start Ignite using configuration from app.config. This closes #417.

Posted by vk...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TypeStringConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TypeStringConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TypeStringConverter.cs
new file mode 100644
index 0000000..e7e8b8f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TypeStringConverter.cs
@@ -0,0 +1,115 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.ComponentModel;
+    using System.Globalization;
+
+    /// <summary>
+    /// Converts string to <see cref="Type"/>.
+    /// </summary>
+    internal class TypeStringConverter : TypeConverter
+    {
+        /// <summary>
+        /// Default instance.
+        /// </summary>
+        public static readonly TypeStringConverter Instance = new TypeStringConverter();
+
+        /// <summary>
+        /// Returns whether this converter can convert an object of the given type to the type of this converter, 
+        /// using the specified context.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="sourceType">A <see cref="Type" /> that represents the type you want to convert from.</param>
+        /// <returns>
+        /// true if this converter can perform the conversion; otherwise, false.
+        /// </returns>
+        public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceType)
+        {
+            return sourceType == typeof (string);
+        }
+
+        /// <summary>
+        /// Returns whether this converter can convert the object to the specified type, using the specified context.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="destinationType">
+        /// A <see cref="Type" /> that represents the type you want to convert to.
+        /// </param>
+        /// <returns>
+        /// true if this converter can perform the conversion; otherwise, false.
+        /// </returns>
+        public override bool CanConvertTo(ITypeDescriptorContext context, Type destinationType)
+        {
+            return destinationType == typeof(string);
+        }
+
+        /// <summary>
+        /// Converts the given object to the type of this converter, 
+        /// using the specified context and culture information.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="culture">The <see cref="CultureInfo" /> to use as the current culture.</param>
+        /// <param name="value">The <see cref="Object" /> to convert.</param>
+        /// <returns>
+        /// An <see cref="Object" /> that represents the converted value.
+        /// </returns>
+        public override object ConvertFrom(ITypeDescriptorContext context, CultureInfo culture, object value)
+        {
+            return value == null ? null : Type.GetType(value.ToString(), false);
+        }
+
+        /// <summary>
+        /// Converts the given value object to the specified type, using the specified context and culture information.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="culture">
+        /// A <see cref="CultureInfo" />. If null is passed, the current culture is assumed.
+        /// </param>
+        /// <param name="value">The <see cref="Object" /> to convert.</param>
+        /// <param name="destinationType">
+        /// The <see cref="Type" /> to convert the <paramref name="value" /> parameter to.
+        /// </param>
+        /// <returns>
+        /// An <see cref="Object" /> that represents the converted value.
+        /// </returns>
+        public override object ConvertTo(ITypeDescriptorContext context, CultureInfo culture, object value, 
+            Type destinationType)
+        {
+            return Convert(value);
+        }
+
+        /// <summary>
+        /// Converts Type to string.
+        /// </summary>
+        /// <param name="value">The value to convert.</param>
+        /// <returns>Resulting string.</returns>
+        public static string Convert(object value)
+        {
+            var type = value as Type;
+            if (type == null)
+                return null;
+
+            if (type.Assembly == typeof (int).Assembly)
+                return type.FullName;
+
+            return type.AssemblyQualifiedName;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
new file mode 100644
index 0000000..6b8f935
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Events
+{
+    using System;
+    using System.Collections.Generic;
+    using System.ComponentModel;
+    using System.Globalization;
+    using System.Linq;
+    using Apache.Ignite.Core.Events;
+
+    /// <summary>
+    /// Converts string to <see cref="EventType"/> member value.
+    /// </summary>
+    internal class EventTypeConverter : TypeConverter
+    {
+        /// <summary>
+        /// Default instance.
+        /// </summary>
+        public static readonly EventTypeConverter Instance = new EventTypeConverter();
+
+        /// <summary>
+        /// The event type map.
+        /// </summary>
+        private static readonly Dictionary<int, string> EvtIdToNameMap =
+            typeof (EventType).GetFields()
+                .Where(p => p.FieldType == typeof (int))
+                .ToDictionary(f => (int) f.GetValue(null), f => f.Name);
+
+        /// <summary>
+        /// The event type map.
+        /// </summary>
+        private static readonly Dictionary<string, int> EvtNameToIdMap =
+            EvtIdToNameMap.ToDictionary(p => p.Value, p => p.Key, StringComparer.OrdinalIgnoreCase);
+
+        /// <summary>
+        /// Returns whether this converter can convert an object of the given type to the type of this converter, 
+        /// using the specified context.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="sourceType">A <see cref="Type" /> that represents the type you want to convert from.</param>
+        /// <returns>
+        /// true if this converter can perform the conversion; otherwise, false.
+        /// </returns>
+        public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceType)
+        {
+            return sourceType == typeof(string);
+        }
+
+        /// <summary>
+        /// Returns whether this converter can convert the object to the specified type, using the specified context.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="destinationType">
+        /// A <see cref="Type" /> that represents the type you want to convert to.
+        /// </param>
+        /// <returns>
+        /// true if this converter can perform the conversion; otherwise, false.
+        /// </returns>
+        public override bool CanConvertTo(ITypeDescriptorContext context, Type destinationType)
+        {
+            return destinationType == typeof(string);
+        }
+
+        /// <summary>
+        /// Converts the given object to the type of this converter, 
+        /// using the specified context and culture information.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="culture">The <see cref="CultureInfo" /> to use as the current culture.</param>
+        /// <param name="value">The <see cref="Object" /> to convert.</param>
+        /// <returns>
+        /// An <see cref="object" /> that represents the converted value.
+        /// </returns>
+        public override object ConvertFrom(ITypeDescriptorContext context, CultureInfo culture, object value)
+        {
+            if (value == null)
+                return null;
+
+            var s = value.ToString();
+            int intResult;
+
+            if (int.TryParse(s, out intResult) || EvtNameToIdMap.TryGetValue(s, out intResult))
+                return intResult;
+
+            throw new InvalidOperationException(string.Format("Cannot convert value to {0}: {1}",
+                typeof (EventType).Name, s));
+        }
+
+        /// <summary>
+        /// Converts the given value object to the specified type, using the specified context and culture information.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="culture">
+        /// A <see cref="CultureInfo" />. If null is passed, the current culture is assumed.
+        /// </param>
+        /// <param name="value">The <see cref="Object" /> to convert.</param>
+        /// <param name="destinationType">
+        /// The <see cref="Type" /> to convert the <paramref name="value" /> parameter to.
+        /// </param>
+        /// <returns>
+        /// An <see cref="object" /> that represents the converted value.
+        /// </returns>
+        public override object ConvertTo(ITypeDescriptorContext context, CultureInfo culture, object value,
+            Type destinationType)
+        {
+            if (!(value is int))
+                return null;
+
+            string eventName;
+
+            if (EvtIdToNameMap.TryGetValue((int)value, out eventName))
+                return eventName;
+
+            return value.ToString();
+        }
+    }
+}
\ No newline at end of file


[22/43] ignite git commit: IGNITE-1563 .NET: Implemented "atomics": AtomicReference and AtomicSequence. This closes #455.

Posted by vk...@apache.org.
IGNITE-1563 .NET: Implemented "atomics": AtomicReference and AtomicSequence. This closes #455.


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

Branch: refs/heads/ignite-2249
Commit: f7c1296cceba73ce1b61af605e476a905a0c8ab4
Parents: e2e216d
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Feb 9 14:43:00 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Feb 9 14:43:00 2016 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicReferenceImpl.java           |  68 ++----
 .../platform/PlatformNoopProcessor.java         |  10 +
 .../processors/platform/PlatformProcessor.java  |  20 ++
 .../platform/PlatformProcessorImpl.java         |  18 ++
 .../callback/PlatformCallbackUtils.java         |   1 -
 .../datastructures/PlatformAtomicReference.java | 141 +++++++++++
 .../datastructures/PlatformAtomicSequence.java  | 122 ++++++++++
 .../cpp/common/include/ignite/common/exports.h  |  15 ++
 .../cpp/common/include/ignite/common/java.h     |  32 +++
 .../platforms/cpp/common/project/vs/module.def  |  15 +-
 modules/platforms/cpp/common/src/exports.cpp    |  52 ++++
 modules/platforms/cpp/common/src/java.cpp       | 181 ++++++++++++++
 .../Apache.Ignite.Core.Tests.csproj             |   2 +
 .../DataStructures/AtomicReferenceTest.cs       | 239 +++++++++++++++++++
 .../DataStructures/AtomicSequenceTest.cs        | 131 ++++++++++
 .../Apache.Ignite.Core.csproj                   |   4 +
 .../DataStructures/IAtomicReference.cs          |  64 +++++
 .../DataStructures/IAtomicSequence.cs           |  69 ++++++
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  28 +++
 .../Impl/DataStructures/AtomicReference.cs      |  92 +++++++
 .../Impl/DataStructures/AtomicSequence.cs       |  90 +++++++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  50 ++++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  12 +
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  37 +++
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  79 ++++++
 25 files changed, 1526 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 37cdaea..e044138 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -35,8 +35,6 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.lang.IgnitePredicate;
 
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -153,10 +151,20 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
 
     /** {@inheritDoc} */
     @Override public boolean compareAndSet(T expVal, T newVal) {
+        return compareAndSetAndGet(newVal, expVal) == expVal;
+    }
+
+    /**
+     * Compares current value with specified value for equality and, if they are equal, replaces current value.
+     *
+     * @param newVal New value to set.
+     * @return Original value.
+     */
+    public T compareAndSetAndGet(T newVal, T expVal) {
         checkRemoved();
 
         try {
-            return CU.outTx(internalCompareAndSet(wrapperPredicate(expVal), wrapperClosure(newVal)), ctx);
+            return CU.outTx(internalCompareAndSetAndGet(expVal, newVal), ctx);
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -197,34 +205,6 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     }
 
     /**
-     * Method make wrapper predicate for existing value.
-     *
-     * @param val Value.
-     * @return Predicate.
-     */
-    private IgnitePredicate<T> wrapperPredicate(final T val) {
-        return new IgnitePredicate<T>() {
-            @Override public boolean apply(T e) {
-                return F.eq(val, e);
-            }
-        };
-    }
-
-    /**
-     * Method make wrapper closure for existing value.
-     *
-     * @param val Value.
-     * @return Closure.
-     */
-    private IgniteClosure<T, T> wrapperClosure(final T val) {
-        return new IgniteClosure<T, T>() {
-            @Override public T apply(T e) {
-                return val;
-            }
-        };
-    }
-
-    /**
      * Method returns callable for execution {@link #set(Object)} operation in async and sync mode.
      *
      * @param val Value will be set in reference .
@@ -260,39 +240,39 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
      * Conditionally sets the new value. It will be set if {@code expValPred} is
      * evaluate to {@code true}.
      *
-     * @param expValPred Predicate which should evaluate to {@code true} for value to be set.
-     * @param newValClos Closure which generates new value.
+     * @param expVal Expected value.
+     * @param newVal New value.
      * @return Callable for execution in async and sync mode.
      */
-    private Callable<Boolean> internalCompareAndSet(final IgnitePredicate<T> expValPred,
-        final IgniteClosure<T, T> newValClos) {
-
-        return retryTopologySafe(new Callable<Boolean>() {
-            @Override public Boolean call() throws Exception {
+    private Callable<T> internalCompareAndSetAndGet(final T expVal, final T newVal) {
+        return retryTopologySafe(new Callable<T>() {
+            @Override public T call() throws Exception {
                 try (IgniteInternalTx tx = CU.txStartInternal(ctx, atomicView, PESSIMISTIC, REPEATABLE_READ)) {
                     GridCacheAtomicReferenceValue<T> ref = atomicView.get(key);
 
                     if (ref == null)
                         throw new IgniteCheckedException("Failed to find atomic reference with given name: " + name);
 
-                    if (!expValPred.apply(ref.get())) {
+                    T origVal = ref.get();
+
+                    if (!F.eq(expVal, origVal)) {
                         tx.setRollbackOnly();
 
-                        return false;
+                        return origVal;
                     }
                     else {
-                        ref.set(newValClos.apply(ref.get()));
+                        ref.set(newVal);
 
                         atomicView.getAndPut(key, ref);
 
                         tx.commit();
 
-                        return true;
+                        return expVal;
                     }
                 }
                 catch (Error | Exception e) {
-                    U.error(log, "Failed to compare and value [expValPred=" + expValPred + ", newValClos" +
-                        newValClos + ", atomicReference" + this + ']', e);
+                    U.error(log, "Failed to compare and value [expVal=" + expVal + ", newVal" +
+                        newVal + ", atomicReference" + this + ']', e);
 
                     throw e;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index b25e32e..8fe17e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -148,4 +148,14 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public void getIgniteConfiguration(long memPtr) {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index b59d93d..2d51c69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -208,6 +208,26 @@ public interface PlatformProcessor extends GridProcessor {
     public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException;
 
     /**
+     * Get or create AtomicSequence.
+     * @param name Name.
+     * @param initVal Initial value.
+     * @param create Create flag.
+     * @return Platform atomic long.
+     * @throws IgniteException
+     */
+    public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException;
+
+    /**
+     * Get or create AtomicReference.
+     * @param name Name.
+     * @param memPtr Pointer to a stream with initial value. 0 for null initial value.
+     * @param create Create flag.
+     * @return Platform atomic long.
+     * @throws IgniteException
+     */
+    public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException;
+
+    /**
      * Gets the configuration of the current Ignite instance.
      *
      * @param memPtr Stream to write data to.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 4ed8c25..d0e0a63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicSequence;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -39,6 +40,8 @@ import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterGro
 import org.apache.ignite.internal.processors.platform.compute.PlatformCompute;
 import org.apache.ignite.internal.processors.platform.datastreamer.PlatformDataStreamer;
 import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicLong;
+import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicReference;
+import org.apache.ignite.internal.processors.platform.datastructures.PlatformAtomicSequence;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore;
 import org.apache.ignite.internal.processors.platform.events.PlatformEvents;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -361,6 +364,21 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
+    @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+        IgniteAtomicSequence atomicSeq = ignite().atomicSequence(name, initVal, create);
+
+        if (atomicSeq == null)
+            return null;
+
+        return new PlatformAtomicSequence(platformCtx, atomicSeq);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
+        return PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+    }
+
+    /** {@inheritDoc} */
     @Override public void getIgniteConfiguration(long memPtr) {
         PlatformOutputStream stream = platformCtx.memory().get(memPtr).output();
         BinaryRawWriterEx writer = platformCtx.writer(stream);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
index 7f3ba6f..3112e0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -442,7 +442,6 @@ public class PlatformCallbackUtils {
     static native void serviceCancel(long envPtr, long svcPtr, long memPtr);
 
     /**
-     /**
      * Invokes service method.
      *
      * @param envPtr Environment pointer.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
new file mode 100644
index 0000000..81b7570
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.datastructures;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicReferenceImpl;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+
+/**
+ * Platform atomic reference wrapper.
+ */
+@SuppressWarnings("unchecked")
+public class PlatformAtomicReference extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_GET = 1;
+
+    /** */
+    private static final int OP_SET = 2;
+
+    /** */
+    private static final int OP_COMPARE_AND_SET_AND_GET = 3;
+
+    /** */
+    private final GridCacheAtomicReferenceImpl atomicRef;
+
+    /**
+     * Creates an instance or returns null.
+     *
+     * @param ctx Context.
+     * @param name Name.
+     * @param memPtr Pointer to a stream with initial value. 0 for default value.
+     * @param create Create flag.
+     * @return Instance of a PlatformAtomicReference, or null when Ignite reference with specific name is null.
+     */
+    public static PlatformAtomicReference createInstance(PlatformContext ctx, String name, long memPtr,
+        boolean create) {
+        assert ctx != null;
+        assert name != null;
+
+        Object initVal = null;
+
+        if (memPtr != 0) {
+            try (PlatformMemory mem = ctx.memory().get(memPtr)) {
+                initVal = ctx.reader(mem).readObjectDetached();
+            }
+        }
+
+        GridCacheAtomicReferenceImpl atomicRef =
+            (GridCacheAtomicReferenceImpl)ctx.kernalContext().grid().atomicReference(name, initVal, create);
+
+        if (atomicRef == null)
+            return null;
+
+        return new PlatformAtomicReference(ctx, atomicRef);
+    }
+
+    /**
+     * Ctor.
+     *
+     * @param ctx Context.
+     * @param ref Atomic reference to wrap.
+     */
+    private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImpl ref) {
+        super(ctx);
+
+        assert ref != null;
+
+        atomicRef = ref;
+    }
+
+    /**
+     * Returns a value indicating whether this instance has been closed.
+     *
+     * @return Value indicating whether this instance has been closed.
+     */
+    public boolean isClosed() {
+        return atomicRef.removed();
+    }
+
+    /**
+     * Closes this instance.
+     */
+    public void close() {
+        atomicRef.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+        if (type == OP_GET)
+            writer.writeObject(atomicRef.get());
+        else
+            super.processOutStream(type, writer);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+        throws IgniteCheckedException {
+        if (type == OP_SET) {
+            atomicRef.set(reader.readObjectDetached());
+
+            return 0;
+        }
+
+        return super.processInStreamOutLong(type, reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+        BinaryRawWriterEx writer) throws IgniteCheckedException {
+        if (type == OP_COMPARE_AND_SET_AND_GET) {
+            Object val = reader.readObjectDetached();
+            final Object cmp = reader.readObjectDetached();
+
+            Object res = atomicRef.compareAndSetAndGet(val, cmp);
+
+            writer.writeObject(res);
+        }
+        else
+            super.processInStreamOutStream(type, reader, writer);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
new file mode 100644
index 0000000..ce7e364
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.datastructures;
+
+import org.apache.ignite.IgniteAtomicSequence;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+
+/**
+ * Platform atomic sequence wrapper.
+ */
+public class PlatformAtomicSequence extends PlatformAbstractTarget {
+    /** */
+    private final IgniteAtomicSequence atomicSeq;
+
+    /**
+     * Ctor.
+     * @param ctx Context.
+     * @param atomicSeq AtomicSequence to wrap.
+     */
+    public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSeq) {
+        super(ctx);
+
+        assert atomicSeq != null;
+
+        this.atomicSeq = atomicSeq;
+    }
+
+    /**
+     * Reads the value.
+     *
+     * @return Current atomic sequence value.
+     */
+    public long get() {
+        return atomicSeq.get();
+    }
+
+    /**
+     * Increments and reads the value.
+     *
+     * @return Current atomic sequence value.
+     */
+    public long incrementAndGet() {
+        return atomicSeq.incrementAndGet();
+    }
+
+    /**
+     * Reads and increments the value.
+     *
+     * @return Original atomic sequence value.
+     */
+    public long getAndIncrement() {
+        return atomicSeq.getAndIncrement();
+    }
+
+    /**
+     * Adds a value.
+     *
+     * @return Current atomic sequence value.
+     */
+    public long addAndGet(long l) {
+        return atomicSeq.addAndGet(l);
+    }
+
+    /**
+     * Adds a value.
+     *
+     * @return Original atomic sequence value.
+     */
+    public long getAndAdd(long l) {
+        return atomicSeq.getAndAdd(l);
+    }
+
+    /**
+     * Gets the batch size.
+     *
+     * @return Batch size.
+     */
+    public int getBatchSize() {
+        return atomicSeq.batchSize();
+    }
+
+    /**
+     * Sets the batch size.
+     *
+     * @param size Batch size.
+     */
+    public void setBatchSize(int size) {
+        atomicSeq.batchSize(size);
+    }
+
+    /**
+     * Gets status of atomic.
+     *
+     * @return {@code true} if atomic was removed from cache, {@code false} in other case.
+     */
+    public boolean isClosed() {
+        return atomicSeq.removed();
+    }
+
+    /**
+     * Removes this atomic.
+     */
+    public void close() {
+        atomicSeq.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/cpp/common/include/ignite/common/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/exports.h b/modules/platforms/cpp/common/include/ignite/common/exports.h
index 66f918f..15911a6 100644
--- a/modules/platforms/cpp/common/include/ignite/common/exports.h
+++ b/modules/platforms/cpp/common/include/ignite/common/exports.h
@@ -48,6 +48,8 @@ extern "C" {
     void* IGNITE_CALL IgniteProcessorServices(gcj::JniContext* ctx, void* obj, void* prj);
     void* IGNITE_CALL IgniteProcessorExtensions(gcj::JniContext* ctx, void* obj);
     void* IGNITE_CALL IgniteProcessorAtomicLong(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
+    void* IGNITE_CALL IgniteProcessorAtomicSequence(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create);
+    void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create);
     void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr);
     
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
@@ -160,6 +162,19 @@ extern "C" {
     bool IGNITE_CALL IgniteAtomicLongIsClosed(gcj::JniContext* ctx, void* obj);
     void IGNITE_CALL IgniteAtomicLongClose(gcj::JniContext* ctx, void* obj);
 
+    long long IGNITE_CALL IgniteAtomicSequenceGet(gcj::JniContext* ctx, void* obj);
+    long long IGNITE_CALL IgniteAtomicSequenceIncrementAndGet(gcj::JniContext* ctx, void* obj);
+    long long IGNITE_CALL IgniteAtomicSequenceGetAndIncrement(gcj::JniContext* ctx, void* obj);
+    long long IGNITE_CALL IgniteAtomicSequenceAddAndGet(gcj::JniContext* ctx, void* obj, long long l);
+    long long IGNITE_CALL IgniteAtomicSequenceGetAndAdd(gcj::JniContext* ctx, void* obj, long long l);
+    int IGNITE_CALL IgniteAtomicSequenceGetBatchSize(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteAtomicSequenceSetBatchSize(gcj::JniContext* ctx, void* obj, int size);
+    bool IGNITE_CALL IgniteAtomicSequenceIsClosed(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteAtomicSequenceClose(gcj::JniContext* ctx, void* obj);
+
+    bool IGNITE_CALL IgniteAtomicReferenceIsClosed(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteAtomicReferenceClose(gcj::JniContext* ctx, void* obj);
+
     bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj);
     bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/cpp/common/include/ignite/common/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/java.h b/modules/platforms/cpp/common/include/ignite/common/java.h
index 072a8ef..8f5823e 100644
--- a/modules/platforms/cpp/common/include/ignite/common/java.h
+++ b/modules/platforms/cpp/common/include/ignite/common/java.h
@@ -311,6 +311,8 @@ namespace ignite
                 jmethodID m_PlatformProcessor_extensions;
                 jmethodID m_PlatformProcessor_atomicLong;
                 jmethodID m_PlatformProcessor_getIgniteConfiguration;
+                jmethodID m_PlatformProcessor_atomicSequence;
+                jmethodID m_PlatformProcessor_atomicReference;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inStreamOutLong;
@@ -353,6 +355,21 @@ namespace ignite
                 jmethodID m_PlatformAtomicLong_isClosed;
                 jmethodID m_PlatformAtomicLong_close;
 
+                jclass c_PlatformAtomicSequence;
+                jmethodID m_PlatformAtomicSequence_get;
+                jmethodID m_PlatformAtomicSequence_incrementAndGet;
+                jmethodID m_PlatformAtomicSequence_getAndIncrement;
+                jmethodID m_PlatformAtomicSequence_addAndGet;
+                jmethodID m_PlatformAtomicSequence_getAndAdd;
+                jmethodID m_PlatformAtomicSequence_getBatchSize;
+                jmethodID m_PlatformAtomicSequence_setBatchSize;
+                jmethodID m_PlatformAtomicSequence_isClosed;
+                jmethodID m_PlatformAtomicSequence_close;
+
+                jclass c_PlatformAtomicReference;
+                jmethodID m_PlatformAtomicReference_isClosed;
+                jmethodID m_PlatformAtomicReference_close;
+
                 jclass c_PlatformListenable;
                 jmethodID m_PlatformListenable_cancel;
                 jmethodID m_PlatformListenable_isCancelled;
@@ -507,6 +524,8 @@ namespace ignite
                 jobject ProcessorServices(jobject obj, jobject prj);
                 jobject ProcessorExtensions(jobject obj);
                 jobject ProcessorAtomicLong(jobject obj, char* name, long long initVal, bool create);
+                jobject ProcessorAtomicSequence(jobject obj, char* name, long long initVal, bool create);
+                jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create);
 				void ProcessorGetIgniteConfiguration(jobject obj, long memPtr);
                 
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
@@ -608,6 +627,19 @@ namespace ignite
                 bool AtomicLongIsClosed(jobject obj);
                 void AtomicLongClose(jobject obj);
 
+                long long AtomicSequenceGet(jobject obj);
+                long long AtomicSequenceIncrementAndGet(jobject obj);
+                long long AtomicSequenceGetAndIncrement(jobject obj);
+                long long AtomicSequenceAddAndGet(jobject obj, long long l);
+                long long AtomicSequenceGetAndAdd(jobject obj, long long l);
+                int AtomicSequenceGetBatchSize(jobject obj);
+                void AtomicSequenceSetBatchSize(jobject obj, int size);
+                bool AtomicSequenceIsClosed(jobject obj);
+                void AtomicSequenceClose(jobject obj);
+
+                bool AtomicReferenceIsClosed(jobject obj);
+                void AtomicReferenceClose(jobject obj);
+
                 bool ListenableCancel(jobject obj);
                 bool ListenableIsCancelled(jobject obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/cpp/common/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/module.def b/modules/platforms/cpp/common/project/vs/module.def
index 81df027..21a4994 100644
--- a/modules/platforms/cpp/common/project/vs/module.def
+++ b/modules/platforms/cpp/common/project/vs/module.def
@@ -116,4 +116,17 @@ IgniteTargetListenFutureForOperationAndGet @113
 IgniteProcessorCreateCacheFromConfig @114
 IgniteProcessorGetOrCreateCacheFromConfig @115
 IgniteProcessorGetIgniteConfiguration @116
-IgniteProcessorDestroyCache @117
\ No newline at end of file
+IgniteProcessorDestroyCache @117
+IgniteProcessorAtomicSequence @118
+IgniteAtomicSequenceGet @119
+IgniteAtomicSequenceIncrementAndGet @120
+IgniteAtomicSequenceGetAndIncrement @121
+IgniteAtomicSequenceAddAndGet @122
+IgniteAtomicSequenceGetAndAdd @123
+IgniteAtomicSequenceGetBatchSize @124
+IgniteAtomicSequenceSetBatchSize @125
+IgniteAtomicSequenceIsClosed @126
+IgniteAtomicSequenceClose @127
+IgniteProcessorAtomicReference @128
+IgniteAtomicReferenceIsClosed @129
+IgniteAtomicReferenceClose @130

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/cpp/common/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/exports.cpp b/modules/platforms/cpp/common/src/exports.cpp
index e9ec519..fff2a16 100644
--- a/modules/platforms/cpp/common/src/exports.cpp
+++ b/modules/platforms/cpp/common/src/exports.cpp
@@ -114,6 +114,14 @@ extern "C" {
         return ctx->ProcessorAtomicLong(static_cast<jobject>(obj), name, initVal, create);
     }
 
+    void* IGNITE_CALL IgniteProcessorAtomicSequence(gcj::JniContext* ctx, void* obj, char* name, long long initVal, bool create) {
+        return ctx->ProcessorAtomicSequence(static_cast<jobject>(obj), name, initVal, create);
+    }
+
+    void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create) {
+        return ctx->ProcessorAtomicReference(static_cast<jobject>(obj), name, memPtr, create);
+    }
+
 	void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long memPtr) {
         return ctx->ProcessorGetIgniteConfiguration(static_cast<jobject>(obj), memPtr);
     }
@@ -482,6 +490,50 @@ extern "C" {
     void IGNITE_CALL IgniteAtomicLongClose(gcj::JniContext* ctx, void* obj) {
         return ctx->AtomicLongClose(static_cast<jobject>(obj));
     }
+
+    long long IGNITE_CALL IgniteAtomicSequenceGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceGet(static_cast<jobject>(obj));
+    }
+
+    long long IGNITE_CALL IgniteAtomicSequenceIncrementAndGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceIncrementAndGet(static_cast<jobject>(obj));
+    }
+
+    long long IGNITE_CALL IgniteAtomicSequenceGetAndIncrement(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceGetAndIncrement(static_cast<jobject>(obj));
+    }
+
+    long long IGNITE_CALL IgniteAtomicSequenceAddAndGet(gcj::JniContext* ctx, void* obj, long long l) {
+        return ctx->AtomicSequenceAddAndGet(static_cast<jobject>(obj), l);
+    }
+
+    long long IGNITE_CALL IgniteAtomicSequenceGetAndAdd(gcj::JniContext* ctx, void* obj, long long l) {
+        return ctx->AtomicSequenceGetAndAdd(static_cast<jobject>(obj), l);
+    }
+
+    int IGNITE_CALL IgniteAtomicSequenceGetBatchSize(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceGetBatchSize(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteAtomicSequenceSetBatchSize(gcj::JniContext* ctx, void* obj, int size) {
+        return ctx->AtomicSequenceSetBatchSize(static_cast<jobject>(obj), size);
+    }
+
+    bool IGNITE_CALL IgniteAtomicSequenceIsClosed(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceIsClosed(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteAtomicSequenceClose(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicSequenceClose(static_cast<jobject>(obj));
+    }
+
+    bool IGNITE_CALL IgniteAtomicReferenceIsClosed(gcj::JniContext* ctx, void* obj) {
+        return ctx->AtomicReferenceIsClosed(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteAtomicReferenceClose(gcj::JniContext* ctx, void* obj) {
+        ctx->AtomicReferenceClose(static_cast<jobject>(obj));
+    }
     
     bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj) {
         return ctx->ListenableCancel(static_cast<jobject>(obj));

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/java.cpp b/modules/platforms/cpp/common/src/java.cpp
index e36c1e0..d6f7ef0 100644
--- a/modules/platforms/cpp/common/src/java.cpp
+++ b/modules/platforms/cpp/common/src/java.cpp
@@ -203,6 +203,8 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
             JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE = JniMethod("atomicSequence", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);            
             JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false);
 
             const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
@@ -396,6 +398,21 @@ namespace ignite
             JniMethod M_PLATFORM_ATOMIC_LONG_IS_CLOSED = JniMethod("isClosed", "()Z", false);
             JniMethod M_PLATFORM_ATOMIC_LONG_CLOSE = JniMethod("close", "()V", false);
 
+            const char* C_PLATFORM_ATOMIC_SEQUENCE = "org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence";
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET = JniMethod("get", "()J", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_INCREMENT_AND_GET = JniMethod("incrementAndGet", "()J", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_INCREMENT = JniMethod("getAndIncrement", "()J", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_ADD_AND_GET = JniMethod("addAndGet", "(J)J", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_ADD = JniMethod("getAndAdd", "(J)J", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_BATCH_SIZE = JniMethod("getBatchSize", "()I", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_SET_BATCH_SIZE = JniMethod("setBatchSize", "(I)V", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_IS_CLOSED = JniMethod("isClosed", "()Z", false);
+            JniMethod M_PLATFORM_ATOMIC_SEQUENCE_CLOSE = JniMethod("close", "()V", false);
+
+            const char* C_PLATFORM_ATOMIC_REFERENCE = "org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference";
+            JniMethod M_PLATFORM_ATOMIC_REFERENCE_IS_CLOSED = JniMethod("isClosed", "()Z", false);
+            JniMethod M_PLATFORM_ATOMIC_REFERENCE_CLOSE = JniMethod("close", "()V", false);
+
             const char* C_PLATFORM_LISTENABLE = "org/apache/ignite/internal/processors/platform/utils/PlatformListenable";
             JniMethod M_PLATFORM_LISTENABLE_CANCEL = JniMethod("cancel", "()Z", false);
             JniMethod M_PLATFORM_LISTENABLE_IS_CANCELED = JniMethod("isCancelled", "()Z", false);
@@ -652,6 +669,8 @@ namespace ignite
                 m_PlatformProcessor_services = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_SERVICES);
                 m_PlatformProcessor_extensions = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_EXTENSIONS);
                 m_PlatformProcessor_atomicLong = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_LONG);
+                m_PlatformProcessor_atomicSequence = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE);
+                m_PlatformProcessor_atomicReference = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE);
 				m_PlatformProcessor_getIgniteConfiguration = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION);
 
                 c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
@@ -695,6 +714,21 @@ namespace ignite
                 m_PlatformAtomicLong_isClosed = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_IS_CLOSED);
                 m_PlatformAtomicLong_close = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_CLOSE);
 
+                jclass c_PlatformAtomicSequence = FindClass(env, C_PLATFORM_ATOMIC_SEQUENCE);
+                m_PlatformAtomicSequence_get = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET);
+                m_PlatformAtomicSequence_incrementAndGet = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_INCREMENT_AND_GET);
+                m_PlatformAtomicSequence_getAndIncrement = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_INCREMENT);
+                m_PlatformAtomicSequence_addAndGet = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_ADD_AND_GET);
+                m_PlatformAtomicSequence_getAndAdd = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_ADD);
+                m_PlatformAtomicSequence_getBatchSize = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_BATCH_SIZE);
+                m_PlatformAtomicSequence_setBatchSize = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_SET_BATCH_SIZE);
+                m_PlatformAtomicSequence_isClosed = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_IS_CLOSED);
+                m_PlatformAtomicSequence_close = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_CLOSE);
+
+                jclass c_PlatformAtomicReference = FindClass(env, C_PLATFORM_ATOMIC_REFERENCE);
+                m_PlatformAtomicReference_isClosed = FindMethod(env, c_PlatformAtomicReference, M_PLATFORM_ATOMIC_REFERENCE_IS_CLOSED);
+                m_PlatformAtomicReference_close = FindMethod(env, c_PlatformAtomicReference, M_PLATFORM_ATOMIC_REFERENCE_CLOSE);
+
                 c_PlatformListenable = FindClass(env, C_PLATFORM_LISTENABLE);
                 m_PlatformListenable_cancel = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_CANCEL);                    
                 m_PlatformListenable_isCancelled = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_IS_CANCELED);
@@ -1307,6 +1341,38 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
+            jobject JniContext::ProcessorAtomicSequence(jobject obj, char* name, long long initVal, bool create)
+            {
+                JNIEnv* env = Attach();
+
+                jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL;
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_atomicSequence, name0, initVal, create);
+
+                if (name0)
+                    env->DeleteLocalRef(name0);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create)
+            {
+                JNIEnv* env = Attach();
+
+                jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL;
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_atomicReference, name0, memPtr, create);
+
+                if (name0)
+                    env->DeleteLocalRef(name0);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
             void JniContext::ProcessorGetIgniteConfiguration(jobject obj, long memPtr)
             {
                 JNIEnv* env = Attach();
@@ -2139,6 +2205,121 @@ namespace ignite
                 ExceptionCheck(env);
             }
 
+            long long JniContext::AtomicSequenceGet(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_get);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            long long JniContext::AtomicSequenceIncrementAndGet(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_incrementAndGet);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            long long JniContext::AtomicSequenceGetAndIncrement(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getAndIncrement);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            long long JniContext::AtomicSequenceAddAndGet(jobject obj, long long l)
+            {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_addAndGet, l);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            long long JniContext::AtomicSequenceGetAndAdd(jobject obj, long long l)
+            {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getAndAdd, l);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            int JniContext::AtomicSequenceGetBatchSize(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getBatchSize);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::AtomicSequenceSetBatchSize(jobject obj, int size)
+            {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_setBatchSize, size);
+
+                ExceptionCheck(env);
+            }
+
+            bool JniContext::AtomicSequenceIsClosed(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_isClosed);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::AtomicSequenceClose(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_close);
+
+                ExceptionCheck(env);
+            }
+
+            bool JniContext::AtomicReferenceIsClosed(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAtomicReference_isClosed);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::AtomicReferenceClose(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicReference_close);
+
+                ExceptionCheck(env);
+            }
+
             bool JniContext::ListenableCancel(jobject obj)
             {
                 JNIEnv* env = Attach();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 481adfb..f5e98c5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -107,6 +107,8 @@
     <Compile Include="Compute\TaskResultTest.cs" />
     <Compile Include="Dataload\DataStreamerTest.cs" />
     <Compile Include="DataStructures\AtomicLongTest.cs" />
+    <Compile Include="DataStructures\AtomicReferenceTest.cs" />
+    <Compile Include="DataStructures\AtomicSequenceTest.cs" />
     <Compile Include="EventsTest.cs" />
     <Compile Include="Examples\Example.cs" />
     <Compile Include="Examples\ExamplesTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicReferenceTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicReferenceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicReferenceTest.cs
new file mode 100644
index 0000000..93375da
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicReferenceTest.cs
@@ -0,0 +1,239 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.DataStructures
+{
+    using System;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Atomic reference test.
+    /// </summary>
+    public class AtomicReferenceTest : IgniteTestBase
+    {
+        /** */
+        private const string AtomicRefName = "testAtomicRef";
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AtomicReferenceTest"/> class.
+        /// </summary>
+        public AtomicReferenceTest() : base("config\\compute\\compute-grid1.xml")
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public override void TestSetUp()
+        {
+            base.TestSetUp();
+
+            // Close test atomic if there is any
+            Grid.GetAtomicReference(AtomicRefName, 0, true).Close();
+        }
+
+        /** <inheritdoc /> */
+        protected override IgniteConfiguration GetConfiguration(string springConfigUrl)
+        {
+            var cfg = base.GetConfiguration(springConfigUrl);
+
+            cfg.BinaryConfiguration = new BinaryConfiguration(typeof(BinaryObj));
+
+            return cfg;
+        }
+
+        /// <summary>
+        /// Tests lifecycle of the AtomicReference.
+        /// </summary>
+        [Test]
+        public void TestCreateClose()
+        {
+            Assert.IsNull(Grid.GetAtomicReference(AtomicRefName, 10, false));
+
+            // Nonexistent atomic returns null
+            Assert.IsNull(Grid.GetAtomicReference(AtomicRefName, 10, false));
+
+            // Create new
+            var al = Grid.GetAtomicReference(AtomicRefName, 10, true);
+            Assert.AreEqual(AtomicRefName, al.Name);
+            Assert.AreEqual(10, al.Read());
+            Assert.AreEqual(false, al.IsClosed);
+
+            // Get existing with create flag
+            var al2 = Grid.GetAtomicReference(AtomicRefName, 5, true);
+            Assert.AreEqual(AtomicRefName, al2.Name);
+            Assert.AreEqual(10, al2.Read());
+            Assert.AreEqual(false, al2.IsClosed);
+
+            // Get existing without create flag
+            var al3 = Grid.GetAtomicReference(AtomicRefName, 5, false);
+            Assert.AreEqual(AtomicRefName, al3.Name);
+            Assert.AreEqual(10, al3.Read());
+            Assert.AreEqual(false, al3.IsClosed);
+
+            al.Close();
+
+            Assert.AreEqual(true, al.IsClosed);
+            Assert.AreEqual(true, al2.IsClosed);
+            Assert.AreEqual(true, al3.IsClosed);
+
+            Assert.IsNull(Grid.GetAtomicReference(AtomicRefName, 10, false));
+        }
+
+        /// <summary>
+        /// Tests modification methods.
+        /// </summary>
+        [Test]
+        public void TestModify()
+        {
+            var atomics = Enumerable.Range(1, 10)
+                .Select(x => Grid.GetAtomicReference(AtomicRefName, 5, true)).ToList();
+
+            atomics.ForEach(x => Assert.AreEqual(5, x.Read()));
+
+            atomics[0].Write(15);
+            atomics.ForEach(x => Assert.AreEqual(15, x.Read()));
+
+            Assert.AreEqual(15, atomics[0].CompareExchange(42, 15));
+            atomics.ForEach(x => Assert.AreEqual(42, x.Read()));
+        }
+
+        /// <summary>
+        /// Tests primitives in the atomic.
+        /// </summary>
+        [Test]
+        public void TestPrimitives()
+        {
+            TestOperations(1, 2);
+            TestOperations("1", "2");
+            TestOperations(Guid.NewGuid(), Guid.NewGuid());
+        }
+
+        /// <summary>
+        /// Tests DateTime in the atomic.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-2578")]
+        public void TestDateTime()
+        {
+            TestOperations(DateTime.Now, DateTime.Now.AddDays(-1));
+        }
+
+        /// <summary>
+        /// Tests serializable objects in the atomic.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-2578")]
+        public void TestSerializable()
+        {
+            TestOperations(new SerializableObj {Foo = 16}, new SerializableObj {Foo = -5});
+        }
+
+        /// <summary>
+        /// Tests binarizable objects in the atomic.
+        /// </summary>
+        [Test]
+        public void TestBinarizable()
+        {
+            TestOperations(new BinaryObj {Foo = 16}, new BinaryObj {Foo = -5});
+        }
+
+        /// <summary>
+        /// Tests operations on specific object.
+        /// </summary>
+        /// <typeparam name="T"></typeparam>
+        /// <param name="x">The x.</param>
+        /// <param name="y">The y.</param>
+        private void TestOperations<T>(T x, T y)
+        {
+            Grid.GetAtomicReference(AtomicRefName, 0, true).Close();
+
+            var atomic = Grid.GetAtomicReference(AtomicRefName, x, true);
+
+            Assert.AreEqual(x, atomic.Read());
+
+            atomic.Write(y);
+            Assert.AreEqual(y, atomic.Read());
+
+            var old = atomic.CompareExchange(x, y);
+            Assert.AreEqual(y, old);
+            Assert.AreEqual(x, atomic.Read());
+
+            old = atomic.CompareExchange(x, y);
+            Assert.AreEqual(x, old);
+            Assert.AreEqual(x, atomic.Read());
+
+            // Check nulls
+            var nul = default(T);
+
+            old = atomic.CompareExchange(nul, x);
+            Assert.AreEqual(x, old);
+            Assert.AreEqual(nul, atomic.Read());
+
+            old = atomic.CompareExchange(y, nul);
+            Assert.AreEqual(nul, old);
+            Assert.AreEqual(y, atomic.Read());
+        }
+
+        /// <summary>
+        /// Serializable.
+        /// </summary>
+        [Serializable]
+        private class SerializableObj
+        {
+            /** */
+            public int Foo { get; set; }
+
+            /** <inheritdoc /> */
+            private bool Equals(SerializableObj other)
+            {
+                return Foo == other.Foo;
+            }
+
+            /** <inheritdoc /> */
+            public override bool Equals(object obj)
+            {
+                if (ReferenceEquals(null, obj)) return false;
+                if (ReferenceEquals(this, obj)) return true;
+                if (obj.GetType() != GetType()) return false;
+                return Equals((SerializableObj) obj);
+            }
+
+            /** <inheritdoc /> */
+            public override int GetHashCode()
+            {
+                // ReSharper disable once NonReadonlyMemberInGetHashCode
+                return Foo;
+            }
+
+            /** <inheritdoc /> */
+            public override string ToString()
+            {
+                return base.ToString() + "[" + Foo + "]";
+            }
+        }
+
+        /// <summary>
+        /// Binary.
+        /// </summary>
+        private sealed class BinaryObj : SerializableObj
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicSequenceTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicSequenceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicSequenceTest.cs
new file mode 100644
index 0000000..472dee2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/DataStructures/AtomicSequenceTest.cs
@@ -0,0 +1,131 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.DataStructures
+{
+    using System.Linq;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Atomic sequence test.
+    /// </summary>
+    public class AtomicSequenceTest : IgniteTestBase
+    {
+        /** */
+        private const string AtomicSeqName = "testAtomicSeq";
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AtomicSequenceTest"/> class.
+        /// </summary>
+        public AtomicSequenceTest() : base("config\\compute\\compute-grid1.xml")
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public override void TestSetUp()
+        {
+            base.TestSetUp();
+
+            // Close test atomic if there is any
+            Grid.GetAtomicSequence(AtomicSeqName, 0, true).Close();
+        }
+
+        /// <summary>
+        /// Tests lifecycle of the AtomicSequence.
+        /// </summary>
+        [Test]
+        public void TestCreateClose()
+        {
+            Assert.IsNull(Grid.GetAtomicSequence(AtomicSeqName, 10, false));
+
+            // Nonexistent atomic returns null
+            Assert.IsNull(Grid.GetAtomicSequence(AtomicSeqName, 10, false));
+
+            // Create new
+            var al = Grid.GetAtomicSequence(AtomicSeqName, 10, true);
+            Assert.AreEqual(AtomicSeqName, al.Name);
+            Assert.AreEqual(10, al.Read());
+            Assert.AreEqual(false, al.IsClosed);
+
+            // Get existing with create flag
+            var al2 = Grid.GetAtomicSequence(AtomicSeqName, 5, true);
+            Assert.AreEqual(AtomicSeqName, al2.Name);
+            Assert.AreEqual(10, al2.Read());
+            Assert.AreEqual(false, al2.IsClosed);
+
+            // Get existing without create flag
+            var al3 = Grid.GetAtomicSequence(AtomicSeqName, 5, false);
+            Assert.AreEqual(AtomicSeqName, al3.Name);
+            Assert.AreEqual(10, al3.Read());
+            Assert.AreEqual(false, al3.IsClosed);
+
+            al.Close();
+
+            Assert.AreEqual(true, al.IsClosed);
+            Assert.AreEqual(true, al2.IsClosed);
+            Assert.AreEqual(true, al3.IsClosed);
+
+            Assert.IsNull(Grid.GetAtomicSequence(AtomicSeqName, 10, false));
+        }
+
+        /// <summary>
+        /// Tests modification methods.
+        /// </summary>
+        [Test]
+        public void TestModify()
+        {
+            var atomics = Enumerable.Range(1, 10)
+                .Select(x => Grid.GetAtomicSequence(AtomicSeqName, 5, true)).ToList();
+
+            atomics.ForEach(x => Assert.AreEqual(5, x.Read()));
+
+            Assert.AreEqual(10, atomics[0].Add(5));
+            atomics.ForEach(x => Assert.AreEqual(10, x.Read()));
+
+            Assert.AreEqual(11, atomics[0].Increment());
+            atomics.ForEach(x => Assert.AreEqual(11, x.Read()));
+
+            atomics.ForEach(x => x.BatchSize = 42);
+            atomics.ForEach(x => Assert.AreEqual(42, x.BatchSize));
+        }
+
+        /// <summary>
+        /// Tests multithreaded scenario.
+        /// </summary>
+        [Test]
+        public void TestMultithreaded()
+        {
+            const int atomicCnt = 10;
+            const int threadCnt = 5;
+            const int iterations = 3000;
+
+            // 10 atomics with same name
+            var atomics = Enumerable.Range(1, atomicCnt)
+                .Select(x => Grid.GetAtomicSequence(AtomicSeqName, 0, true)).ToList();
+
+            // 5 threads increment 30000 times
+            TestUtils.RunMultiThreaded(() =>
+            {
+                for (var i = 0; i < iterations; i++)
+                    atomics.ForEach(x => x.Increment());
+            }, threadCnt);
+
+            atomics.ForEach(x => Assert.AreEqual(atomicCnt*threadCnt*iterations, x.Read()));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 1c83168..e2efd0a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -161,6 +161,8 @@
     <Compile Include="Datastream\StreamTransformer.cs" />
     <Compile Include="Datastream\StreamVisitor.cs" />
     <Compile Include="DataStructures\IAtomicLong.cs" />
+    <Compile Include="DataStructures\IAtomicReference.cs" />
+    <Compile Include="DataStructures\IAtomicSequence.cs" />
     <Compile Include="DataStructures\Package-Info.cs" />
     <Compile Include="Events\CacheEvent.cs" />
     <Compile Include="Events\CacheQueryExecutedEvent.cs" />
@@ -259,6 +261,8 @@
     <Compile Include="Impl\Datastream\DataStreamerRemoveEntry.cs" />
     <Compile Include="Impl\Datastream\StreamReceiverHolder.cs" />
     <Compile Include="Impl\DataStructures\AtomicLong.cs" />
+    <Compile Include="Impl\DataStructures\AtomicReference.cs" />
+    <Compile Include="Impl\DataStructures\AtomicSequence.cs" />
     <Compile Include="Impl\Events\Events.cs" />
     <Compile Include="Impl\Events\RemoteListenEventFilter.cs" />
     <Compile Include="Impl\ExceptionUtils.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicReference.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicReference.cs b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicReference.cs
new file mode 100644
index 0000000..403c0ca
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicReference.cs
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.DataStructures
+{
+    /// <summary>
+    /// Represents a named value in the distributed cache.
+    /// </summary>
+    public interface IAtomicReference<T>
+    {
+        /// <summary>
+        /// Gets the name of this atomic reference.
+        /// </summary>
+        /// <value>
+        /// Name of this atomic reference.
+        /// </value>
+        string Name { get; }
+
+        /// <summary>
+        /// Reads current value of an atomic reference.
+        /// </summary>
+        /// <returns>Current value of an atomic reference.</returns>
+        T Read();
+
+        /// <summary>
+        /// Writes current value of an atomic reference.
+        /// </summary>
+        /// <param name="value">The value to set.</param>
+        void Write(T value);
+
+        /// <summary>
+        /// Compares current value with specified value for equality and, if they are equal, replaces current value.
+        /// </summary>
+        /// <param name="value">The value to set.</param>
+        /// <param name="comparand">The value that is compared to the current value.</param>
+        /// <returns>Original value of the atomic reference.</returns>
+        T CompareExchange(T value, T comparand);
+
+        /// <summary>
+        /// Determines whether this instance was removed from cache.
+        /// </summary>
+        /// <returns>True if this atomic was removed from cache; otherwise, false.</returns>
+        bool IsClosed { get; }
+
+        /// <summary>
+        /// Closes this instance.
+        /// </summary>
+        void Close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicSequence.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicSequence.cs b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicSequence.cs
new file mode 100644
index 0000000..f5b1dad
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/IAtomicSequence.cs
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.DataStructures
+{
+    /// <summary>
+    /// Represents a distributed atomic sequence of numbers.
+    /// </summary>
+    public interface IAtomicSequence
+    {
+        /// <summary>
+        /// Gets the name of this atomic sequence.
+        /// </summary>
+        /// <value>
+        /// Name of this atomic sequence.
+        /// </value>
+        string Name { get; }
+
+        /// <summary>
+        /// Returns current value.
+        /// </summary>
+        /// <returns>Current value of the atomic sequence.</returns>
+        long Read();
+
+        /// <summary>
+        /// Increments current value and returns result.
+        /// </summary>
+        /// <returns>The new value of the atomic sequence.</returns>
+        long Increment();
+
+        /// <summary>
+        /// Adds specified value to the current value and returns result.
+        /// </summary>
+        /// <param name="value">The value to add.</param>
+        /// <returns>The new value of the atomic sequence.</returns>
+        long Add(long value);
+
+        /// <summary>
+        /// Gets local batch size for this atomic sequence.
+        /// </summary>
+        /// <returns>Sequence batch size.</returns>
+        int BatchSize { get; set; }
+
+        /// <summary>
+        /// Determines whether this instance was removed from cache.
+        /// </summary>
+        /// <returns>True if this atomic was removed from cache; otherwise, false.</returns>
+        bool IsClosed { get; }
+
+        /// <summary>
+        /// Closes this instance.
+        /// </summary>
+        void Close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
index d18e790..12ea09e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -192,6 +192,34 @@ namespace Apache.Ignite.Core
         IAtomicLong GetAtomicLong(string name, long initialValue, bool create);
 
         /// <summary>
+        /// Gets an atomic sequence with specified name from cache.
+        /// Creates new atomic sequence in cache if it does not exist and <paramref name="create"/> is true.
+        /// </summary>
+        /// <param name="name">Name of the atomic sequence.</param>
+        /// <param name="initialValue">
+        /// Initial value for the atomic sequence. Ignored if <paramref name="create"/> is false.
+        /// </param>
+        /// <param name="create">Flag indicating whether atomic sequence should be created if it does not exist.</param>
+        /// <returns>Atomic sequence instance with specified name, 
+        /// or null if it does not exist and <paramref name="create"/> flag is not set.</returns>
+        /// <exception cref="IgniteException">If atomic sequence could not be fetched or created.</exception>
+        IAtomicSequence GetAtomicSequence(string name, long initialValue, bool create);
+
+        /// <summary>
+        /// Gets an atomic reference with specified name from cache.
+        /// Creates new atomic reference in cache if it does not exist and <paramref name="create"/> is true.
+        /// </summary>
+        /// <param name="name">Name of the atomic reference.</param>
+        /// <param name="initialValue">
+        /// Initial value for the atomic reference. Ignored if <paramref name="create"/> is false.
+        /// </param>
+        /// <param name="create">Flag indicating whether atomic reference should be created if it does not exist.</param>
+        /// <returns>Atomic reference instance with specified name, 
+        /// or null if it does not exist and <paramref name="create"/> flag is not set.</returns>
+        /// <exception cref="IgniteException">If atomic reference could not be fetched or created.</exception>
+        IAtomicReference<T> GetAtomicReference<T>(string name, T initialValue, bool create);
+
+        /// <summary>
         /// Gets the configuration of this Ignite instance.
         /// </summary>
         [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs
new file mode 100644
index 0000000..e871412
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs
@@ -0,0 +1,92 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.DataStructures
+{
+    using System.Diagnostics;
+    using Apache.Ignite.Core.DataStructures;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Atomic reference.
+    /// </summary>
+    internal class AtomicReference<T> : PlatformTarget, IAtomicReference<T>
+    {
+        /** Opcodes. */
+        private enum Op
+        {
+            Get = 1,
+            Set = 2,
+            CompareAndSetAndGet = 3
+        }
+
+        /** */
+        private readonly string _name;
+
+        /** <inheritDoc /> */
+        public AtomicReference(IUnmanagedTarget target, Marshaller marsh, string name)
+            : base(target, marsh)
+        {
+            Debug.Assert(!string.IsNullOrEmpty(name));
+
+            _name = name;
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return _name; }
+        }
+
+        /** <inheritDoc /> */
+        public T Read()
+        {
+            return DoInOp<T>((int) Op.Get);
+        }
+
+        /** <inheritDoc /> */
+        public void Write(T value)
+        {
+            DoOutOp((int) Op.Set, value);
+        }
+
+        /** <inheritDoc /> */
+        public T CompareExchange(T value, T comparand)
+        {
+            return DoOutInOp((int) Op.CompareAndSetAndGet,
+                writer =>
+                {
+                    writer.WriteObject(value);
+                    writer.WriteObject(comparand);
+                },
+                stream => Marshaller.StartUnmarshal(stream).Deserialize<T>());
+        }
+
+        /** <inheritDoc /> */
+        public bool IsClosed
+        {
+            get { return UnmanagedUtils.AtomicReferenceIsClosed(Target); }
+        }
+
+        /** <inheritDoc /> */
+        public void Close()
+        {
+            UnmanagedUtils.AtomicReferenceClose(Target);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs
new file mode 100644
index 0000000..0835b9a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs
@@ -0,0 +1,90 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.DataStructures
+{
+    using System.Diagnostics;
+    using Apache.Ignite.Core.DataStructures;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Atomic long wrapper.
+    /// </summary>
+    internal sealed class AtomicSequence: PlatformTarget, IAtomicSequence
+    {
+        /** */
+        private readonly string _name;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Apache.Ignite.Core.Impl.DataStructures.AtomicLong"/> class.
+        /// </summary>
+        /// <param name="target">The target.</param>
+        /// <param name="marsh">The marshaller.</param>
+        /// <param name="name">The name.</param>
+        public AtomicSequence(IUnmanagedTarget target, Marshaller marsh, string name)
+            : base(target, marsh)
+        {
+            Debug.Assert(!string.IsNullOrEmpty(name));
+
+            _name = name;
+        }
+
+        /** <inheritDoc /> */
+        public string Name
+        {
+            get { return _name; }
+        }
+
+        /** <inheritDoc /> */
+        public long Read()
+        {
+            return UnmanagedUtils.AtomicSequenceGet(Target);
+        }
+
+        /** <inheritDoc /> */
+        public long Increment()
+        {
+            return UnmanagedUtils.AtomicSequenceIncrementAndGet(Target);
+        }
+
+        /** <inheritDoc /> */
+        public long Add(long value)
+        {
+            return UnmanagedUtils.AtomicSequenceAddAndGet(Target, value);
+        }
+
+        /** <inheritDoc /> */
+        public int BatchSize
+        {
+            get { return UnmanagedUtils.AtomicSequenceGetBatchSize(Target); }
+            set { UnmanagedUtils.AtomicSequenceSetBatchSize(Target, value); }
+        }
+
+        /** <inheritDoc /> */
+        public bool IsClosed
+        {
+            get { return UnmanagedUtils.AtomicSequenceIsClosed(Target); }
+        }
+
+        /** <inheritDoc /> */
+        public void Close()
+        {
+            UnmanagedUtils.AtomicSequenceClose(Target);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index 9d27117..be21d7f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -486,6 +486,56 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        public IAtomicSequence GetAtomicSequence(string name, long initialValue, bool create)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            var nativeSeq = UU.ProcessorAtomicSequence(_proc, name, initialValue, create);
+
+            if (nativeSeq == null)
+                return null;
+
+            return new AtomicSequence(nativeSeq, Marshaller, name);
+        }
+
+        /** <inheritdoc /> */
+        public IAtomicReference<T> GetAtomicReference<T>(string name, T initialValue, bool create)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            var refTarget = GetAtomicReferenceUnmanaged(name, initialValue, create);
+
+            return refTarget == null ? null : new AtomicReference<T>(refTarget, Marshaller, name);
+        }
+
+        /// <summary>
+        /// Gets the unmanaged atomic reference.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <param name="initialValue">The initial value.</param>
+        /// <param name="create">Create flag.</param>
+        /// <returns>Unmanaged atomic reference, or null.</returns>
+        private IUnmanagedTarget GetAtomicReferenceUnmanaged<T>(string name, T initialValue, bool create)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            // Do not allocate memory when default is not used.
+            if (!create)
+                return UU.ProcessorAtomicReference(_proc, name, 0, false);
+            
+            using (var stream = IgniteManager.Memory.Allocate().GetStream())
+            {
+                var writer = Marshaller.StartMarshal(stream);
+
+                writer.Write(initialValue);
+
+                var memPtr = stream.SynchronizeOutput();
+
+                return UU.ProcessorAtomicReference(_proc, name, memPtr, true);
+            }
+        }
+
+        /** <inheritdoc /> */
         public IgniteConfiguration GetConfiguration()
         {
             using (var stream = IgniteManager.Memory.Allocate(1024).GetStream())

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
index 46bc3ca..a303783 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -340,6 +340,18 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        public IAtomicSequence GetAtomicSequence(string name, long initialValue, bool create)
+        {
+            return _ignite.GetAtomicSequence(name, initialValue, create);
+        }
+
+        /** <inheritdoc /> */
+        public IAtomicReference<T> GetAtomicReference<T>(string name, T initialValue, bool create)
+        {
+            return _ignite.GetAtomicReference(name, initialValue, create);
+        }
+
+        /** <inheritdoc /> */
         public void WriteBinary(IBinaryWriter writer)
         {
             // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
index 17df94a..28eb208 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -93,6 +93,14 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         public static extern void* ProcessorAtomicLong(void* ctx, void* obj, sbyte* name, long initVal,
             [MarshalAs(UnmanagedType.U1)] bool create);
 
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorAtomicSequence")]
+        public static extern void* ProcessorAtomicSequence(void* ctx, void* obj, sbyte* name, long initVal,
+            [MarshalAs(UnmanagedType.U1)] bool create);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorAtomicReference")]
+        public static extern void* ProcessorAtomicReference(void* ctx, void* obj, sbyte* name, long memPtr,
+            [MarshalAs(UnmanagedType.U1)] bool create);
+
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetIgniteConfiguration")]
         public static extern void ProcessorGetIgniteConfiguration(void* ctx, void* obj, long memPtr);
 
@@ -373,6 +381,35 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongClose")]
         public static extern void AtomicLongClose(void* ctx, void* target);
 
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceGet")]
+        public static extern long AtomicSequenceGet(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceIncrementAndGet")]
+        public static extern long AtomicSequenceIncrementAndGet(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceAddAndGet")]
+        public static extern long AtomicSequenceAddAndGet(void* ctx, void* target, long value);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceGetBatchSize")]
+        public static extern int AtomicSequenceGetBatchSize(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceSetBatchSize")]
+        public static extern void AtomicSequenceSetBatchSize(void* ctx, void* target, int size);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceIsClosed")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool AtomicSequenceIsClosed(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceClose")]
+        public static extern void AtomicSequenceClose(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicReferenceIsClosed")]
+        [return: MarshalAs(UnmanagedType.U1)]
+        public static extern bool AtomicReferenceIsClosed(void* ctx, void* target);
+
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicReferenceClose")]
+        public static extern void AtomicReferenceClose(void* ctx, void* target);
+
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableCancel")]
         [return: MarshalAs(UnmanagedType.U1)]
         public static extern bool ListenableCancel(void* ctx, void* target);


[43/43] ignite git commit: IGNITE-2249 - Do not deserialize services on client node

Posted by vk...@apache.org.
IGNITE-2249 - Do not deserialize services on client node


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

Branch: refs/heads/ignite-2249
Commit: 798e570bb10463d93c2ec4427fa586f3384ac0a7
Parents: 6ff7391
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Feb 11 15:27:11 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Feb 11 15:27:11 2016 -0800

----------------------------------------------------------------------
 .../internal/processors/service/GridServiceProcessor.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/798e570b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index a1bf11f..1a48e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -173,11 +173,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             if (ctx.deploy().enabled())
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
+            boolean affNode = cache.context().affinityNode();
+
             cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, cache.context().affinityNode(), true, true);
+                new DeploymentListener(), null, affNode, true, !affNode);
 
             assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, cache.context().affinityNode(), true, true);
+                new AssignmentListener(), null, affNode, true, !affNode);
         }
         finally {
             if (ctx.deploy().enabled())
@@ -1535,7 +1537,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         private static final long serialVersionUID = 0L;
 
         /** */
-        private static final IgniteProductVersion SINCE_VER = IgniteProductVersion.fromString("1.5.0");
+        private static final IgniteProductVersion SINCE_VER = IgniteProductVersion.fromString("1.5.7");
 
         /** */
         private final String svcName;


[11/43] ignite git commit: Fixed code style.

Posted by vk...@apache.org.
Fixed code style.


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

Branch: refs/heads/ignite-2249
Commit: bad042097f5030c6eef31f4a24e987876ae9176e
Parents: e88cc67
Author: sboikov <sb...@gridgain.com>
Authored: Mon Feb 8 09:47:27 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Feb 8 09:47:27 2016 +0300

----------------------------------------------------------------------
 .../util/nio/SelectedSelectionKeySet.java       | 65 +++++++++++++-------
 1 file changed, 43 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bad04209/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
index 9aa245d..389975a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
@@ -21,63 +21,85 @@ import java.nio.channels.SelectionKey;
 import java.util.AbstractSet;
 import java.util.Iterator;
 
+/**
+ *
+ */
 final class SelectedSelectionKeySet extends AbstractSet<SelectionKey> {
-
+    /** */
     private SelectionKey[] keysA;
+
+    /** */
     private int keysASize;
+
+    /** */
     private SelectionKey[] keysB;
+
+    /** */
     private int keysBSize;
+
+    /** */
     private boolean isA = true;
 
+    /**
+     *
+     */
     SelectedSelectionKeySet() {
         keysA = new SelectionKey[1024];
         keysB = keysA.clone();
     }
 
-    @Override
-    public boolean add(SelectionKey o) {
-        if (o == null) {
+    /** {@inheritDoc} */
+    @Override public boolean add(SelectionKey o) {
+        if (o == null)
             return false;
-        }
 
         if (isA) {
             int size = keysASize;
             keysA[size ++] = o;
             keysASize = size;
-            if (size == keysA.length) {
+            if (size == keysA.length)
                 doubleCapacityA();
-            }
-        } else {
+        }
+        else {
             int size = keysBSize;
             keysB[size ++] = o;
             keysBSize = size;
-            if (size == keysB.length) {
+            if (size == keysB.length)
                 doubleCapacityB();
-            }
         }
 
         return true;
     }
 
+    /**
+     *
+     */
     private void doubleCapacityA() {
         SelectionKey[] newKeysA = new SelectionKey[keysA.length << 1];
         System.arraycopy(keysA, 0, newKeysA, 0, keysASize);
         keysA = newKeysA;
     }
 
+    /**
+     *
+     */
     private void doubleCapacityB() {
         SelectionKey[] newKeysB = new SelectionKey[keysB.length << 1];
         System.arraycopy(keysB, 0, newKeysB, 0, keysBSize);
         keysB = newKeysB;
     }
 
+    /**
+     * @return Selection keys.
+     */
     SelectionKey[] flip() {
         if (isA) {
             isA = false;
             keysA[keysASize] = null;
             keysBSize = 0;
             return keysA;
-        } else {
+        }
+        else {
             isA = true;
             keysB[keysBSize] = null;
             keysASize = 0;
@@ -85,27 +107,26 @@ final class SelectedSelectionKeySet extends AbstractSet<SelectionKey> {
         }
     }
 
-    @Override
-    public int size() {
-        if (isA) {
+    /** {@inheritDoc} */
+    @Override public int size() {
+        if (isA)
             return keysASize;
-        } else {
+        else
             return keysBSize;
-        }
     }
 
-    @Override
-    public boolean remove(Object o) {
+    /** {@inheritDoc} */
+    @Override public boolean remove(Object o) {
         return false;
     }
 
-    @Override
-    public boolean contains(Object o) {
+    /** {@inheritDoc} */
+    @Override public boolean contains(Object o) {
         return false;
     }
 
-    @Override
-    public Iterator<SelectionKey> iterator() {
+    /** {@inheritDoc} */
+    @Override public Iterator<SelectionKey> iterator() {
         throw new UnsupportedOperationException();
     }
 }


[25/43] ignite git commit: IGNITE-2575: Added validation of IGFS endpoint port value. This closes #469.

Posted by vk...@apache.org.
IGNITE-2575: Added validation of IGFS endpoint port value. This closes #469.


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

Branch: refs/heads/ignite-2249
Commit: b7475f09b1727e5cc93681ee229b60ad8e188732
Parents: a4d8a04
Author: dkarachentsev <dk...@gridgain.com>
Authored: Wed Feb 10 12:38:43 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 12:38:43 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsProcessor.java | 14 ++++++++++
 .../igfs/IgfsProcessorValidationSelfTest.java   | 27 ++++++++++++++++++++
 2 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7475f09/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 21446e1..1b60252 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -67,6 +67,12 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     /** Null IGFS name. */
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
+    /** Min available TCP port. */
+    private static final int MIN_TCP_PORT = 1;
+
+    /** Max available TCP port. */
+    private static final int MAX_TCP_PORT = 0xFFFF;
+
     /** Converts context to IGFS. */
     private static final IgniteClosure<IgfsContext,IgniteFileSystem> CTX_TO_IGFS = new C1<IgfsContext, IgniteFileSystem>() {
         @Override public IgniteFileSystem apply(IgfsContext igfsCtx) {
@@ -307,6 +313,14 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
                 throw new IgniteCheckedException("Invalid IGFS data cache configuration (key affinity mapper class should be " +
                     IgfsGroupDataBlocksKeyMapper.class.getSimpleName() + "): " + cfg);
 
+            if (cfg.getIpcEndpointConfiguration() != null) {
+                final int tcpPort = cfg.getIpcEndpointConfiguration().getPort();
+
+                if (!(tcpPort >= MIN_TCP_PORT && tcpPort <= MAX_TCP_PORT))
+                    throw new IgniteCheckedException("IGFS endpoint TCP port is out of range [" + MIN_TCP_PORT +
+                        ".." + MAX_TCP_PORT + "]: " + tcpPort);
+            }
+
             long maxSpaceSize = cfg.getMaxSpaceSize();
 
             if (maxSpaceSize > 0) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7475f09/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
index 11a80af..27f47e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
+import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
@@ -442,6 +443,32 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidEndpointTcpPort() throws Exception {
+        final String failMsg = "IGFS endpoint TCP port is out of range";
+        g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
+
+        final String igfsCfgName = "igfs-cfg";
+        final IgfsIpcEndpointConfiguration igfsEndpointCfg = new IgfsIpcEndpointConfiguration();
+        igfsEndpointCfg.setPort(0);
+        g1IgfsCfg1.setName(igfsCfgName);
+        g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
+
+        checkGridStartFails(g1Cfg, failMsg, true);
+
+        igfsEndpointCfg.setPort(-1);
+        g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
+
+        checkGridStartFails(g1Cfg, failMsg, true);
+
+        igfsEndpointCfg.setPort(65536);
+        g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
+
+        checkGridStartFails(g1Cfg, failMsg, true);
+    }
+
+    /**
      * Checks that the given grid configuration will lead to {@link IgniteCheckedException} upon grid startup.
      *
      * @param cfg Grid configuration to check.


[06/43] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vk...@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/cc5067bc
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/cc5067bc
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/cc5067bc

Branch: refs/heads/ignite-2249
Commit: cc5067bc5a14a711f6c197c7d48069bf445893b7
Parents: 3602d46 88b0eeb
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Feb 5 13:02:36 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Feb 5 13:02:36 2016 +0300

----------------------------------------------------------------------
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |   3 +
 .../benchmarks/jmh/cache/JmhCacheBenchmark.java | 145 +++++++++++++++++++
 .../jmh/cache/JmhCachePutBenchmark.java         | 124 ----------------
 .../jmh/runner/JmhIdeBenchmarkRunner.java       |  20 ++-
 .../affinity/GridAffinityAssignment.java        |  36 ++++-
 .../affinity/GridAffinityAssignmentCache.java   |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  10 +-
 .../cache/GridCacheAffinityManager.java         |  30 +++-
 .../dht/GridDhtPartitionTopologyImpl.java       |   9 +-
 .../osgi-karaf/src/main/resources/features.xml  |  12 +-
 10 files changed, 238 insertions(+), 153 deletions(-)
----------------------------------------------------------------------



[38/43] ignite git commit: Improved KerberosHadoopFileSystemFactory JavaDocs.

Posted by vk...@apache.org.
Improved KerberosHadoopFileSystemFactory JavaDocs.


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

Branch: refs/heads/ignite-2249
Commit: a9937a6e20407189b8bd67b2cc30a30ddc8dd6ce
Parents: d08a779
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 10 16:57:16 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 16:57:16 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9937a6e/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
index fc768d6..a78cabc 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
@@ -106,6 +106,9 @@ public class KerberosHadoopFileSystemFactory extends BasicHadoopFileSystemFactor
 
     /**
      * Gets the key tab full file name (e.g. "/etc/security/keytabs/hdfs.headless.keytab" or "/etc/krb5.keytab").
+     * <p>
+     * <b>NOTE!</b> Factory can be serialized and transferred to other machines where instance of
+     * {@link IgniteHadoopFileSystem} resides. Corresponding path must exist on these machines as well.
      *
      * @return The key tab file name.
      */


[10/43] ignite git commit: IGNITE-2450 - Fixed java.lang.reflect.Proxy serialization

Posted by vk...@apache.org.
IGNITE-2450 - Fixed java.lang.reflect.Proxy serialization


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

Branch: refs/heads/ignite-2249
Commit: e88cc676fea65087edb698ff8557de329c344f71
Parents: 75961ee
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Feb 5 12:55:27 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Feb 5 12:55:27 2016 -0800

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  |  22 ++++
 .../internal/binary/BinaryReaderExImpl.java     |   8 +-
 .../ignite/internal/binary/BinaryUtils.java     |  48 ++++---
 .../ignite/internal/binary/BinaryWriteMode.java |   3 +
 .../internal/binary/BinaryWriterExImpl.java     |  34 +++++
 .../internal/binary/GridBinaryMarshaller.java   |   3 +
 .../optimized/OptimizedClassDescriptor.java     |  37 +++++-
 .../optimized/OptimizedMarshallerUtils.java     |   3 +
 .../optimized/OptimizedObjectInputStream.java   |  15 ++-
 .../optimized/OptimizedObjectOutputStream.java  |   4 +-
 ...namicProxySerializationMultiJvmSelfTest.java | 131 +++++++++++++++++++
 .../junits/multijvm/IgniteNodeRunner.java       |  16 ++-
 .../junits/multijvm/IgniteProcessProxy.java     |  19 +--
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   2 +
 15 files changed, 313 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 5cb8a86..e831e96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -22,6 +22,7 @@ import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -109,6 +110,9 @@ public class BinaryClassDescriptor {
     /** */
     private final boolean excluded;
 
+    /** */
+    private final Class<?>[] intfs;
+
     /**
      * @param ctx Context.
      * @param cls Class.
@@ -229,6 +233,16 @@ public class BinaryClassDescriptor {
                 fields = null;
                 stableFieldsMeta = null;
                 stableSchema = null;
+                intfs = null;
+
+                break;
+
+            case PROXY:
+                ctor = null;
+                fields = null;
+                stableFieldsMeta = null;
+                stableSchema = null;
+                intfs = cls.getInterfaces();
 
                 break;
 
@@ -237,6 +251,7 @@ public class BinaryClassDescriptor {
                 fields = null;
                 stableFieldsMeta = null;
                 stableSchema = null;
+                intfs = null;
 
                 break;
 
@@ -291,6 +306,8 @@ public class BinaryClassDescriptor {
 
                 stableSchema = schemaBuilder.build();
 
+                intfs = null;
+
                 break;
 
             default:
@@ -611,6 +628,11 @@ public class BinaryClassDescriptor {
 
                 break;
 
+            case PROXY:
+                writer.doWriteProxy((Proxy)obj, intfs);
+
+                break;
+
             case BINARY_OBJ:
                 writer.doWriteBinaryObject((BinaryObjectImpl)obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
index 607dabc..f9e7aa5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java
@@ -68,6 +68,7 @@ import static org.apache.ignite.internal.binary.GridBinaryMarshaller.NULL;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OBJ_ARR;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.OPTM_MARSH;
+import static org.apache.ignite.internal.binary.GridBinaryMarshaller.PROXY;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.SHORT_ARR;
 import static org.apache.ignite.internal.binary.GridBinaryMarshaller.STRING;
@@ -1621,6 +1622,11 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
 
                 break;
 
+            case PROXY:
+                obj = BinaryUtils.doReadProxy(in, ctx, ldr, this);
+
+                break;
+
             case OPTM_MARSH:
                 obj = BinaryUtils.doReadOptimized(in, ctx, ldr);
 
@@ -2013,7 +2019,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Bina
     /**
      * Flag.
      */
-    private static enum Flag {
+    private enum Flag {
         /** Regular. */
         NORMAL,
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index a82b65f..c3343d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -17,28 +17,15 @@
 
 package org.apache.ignite.internal.binary;
 
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryCollectionFactory;
-import org.apache.ignite.binary.BinaryInvalidTypeException;
-import org.apache.ignite.binary.BinaryMapFactory;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.internal.binary.builder.BinaryLazyValue;
-import org.apache.ignite.internal.binary.streams.BinaryInputStream;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
-
 import java.io.ByteArrayInputStream;
 import java.io.Externalizable;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.lang.reflect.Array;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -58,6 +45,20 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryCollectionFactory;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryMapFactory;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.binary.builder.BinaryLazyValue;
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -1042,6 +1043,8 @@ public class BinaryUtils {
             return BinaryWriteMode.ENUM;
         else if (cls == Class.class)
             return BinaryWriteMode.CLASS;
+        else if (Proxy.class.isAssignableFrom(cls))
+            return BinaryWriteMode.PROXY;
         else
             return BinaryWriteMode.OBJECT;
     }
@@ -1365,6 +1368,21 @@ public class BinaryUtils {
     }
 
     /**
+     * @return Value.
+     */
+    public static Object doReadProxy(BinaryInputStream in, BinaryContext ctx, ClassLoader ldr,
+        BinaryReaderHandlesHolder handles) {
+        Class<?>[] intfs = new Class<?>[in.readInt()];
+
+        for (int i = 0; i < intfs.length; i++)
+            intfs[i] = doReadClass(in, ctx, ldr);
+
+        InvocationHandler ih = (InvocationHandler)doReadObject(in, ctx, ldr, handles);
+
+        return Proxy.newProxyInstance(ldr != null ? ldr : U.gridClassLoader(), intfs, ih);
+    }
+
+    /**
      * Read plain type.
      *
      * @param in Input stream.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
index b037945..7e8c9bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
@@ -148,6 +148,9 @@ public enum BinaryWriteMode {
     CLASS(GridBinaryMarshaller.CLASS),
 
     /** */
+    PROXY(GridBinaryMarshaller.PROXY),
+
+    /** */
     BINARY(GridBinaryMarshaller.BINARY_OBJ),
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
index 877a2db..8060a13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java
@@ -19,7 +19,9 @@ package org.apache.ignite.internal.binary;
 
 import java.io.IOException;
 import java.io.ObjectOutput;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Proxy;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -817,6 +819,38 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje
     }
 
     /**
+     * @param proxy Proxy.
+     */
+    public void doWriteProxy(Proxy proxy, Class<?>[] intfs) {
+        if (proxy == null)
+            out.writeByte(GridBinaryMarshaller.NULL);
+        else {
+            out.unsafeEnsure(1 + 4);
+
+            out.unsafeWriteByte(GridBinaryMarshaller.PROXY);
+            out.unsafeWriteInt(intfs.length);
+
+            for (Class<?> intf : intfs) {
+                BinaryClassDescriptor desc = ctx.descriptorForClass(intf, false);
+
+                if (desc.registered())
+                    out.writeInt(desc.typeId());
+                else {
+                    out.writeInt(GridBinaryMarshaller.UNREGISTERED_TYPE_ID);
+
+                    doWriteString(intf.getName());
+                }
+            }
+
+            InvocationHandler ih = Proxy.getInvocationHandler(proxy);
+
+            assert ih != null;
+
+            doWriteObject(ih);
+        }
+    }
+
+    /**
      * @param po Binary object.
      */
     public void doWriteBinaryObject(@Nullable BinaryObjectImpl po) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
index 8e138fc..67e741b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
@@ -144,6 +144,9 @@ public class GridBinaryMarshaller {
     /** Timestamp array. */
     public static final byte TIMESTAMP_ARR = 34;
 
+    /** Proxy. */
+    public static final byte PROXY = 35;
+
     /** */
     public static final byte NULL = (byte)101;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index c5be139..5a5b54d 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -26,9 +26,11 @@ import java.io.ObjectStreamField;
 import java.io.Serializable;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -79,6 +81,7 @@ import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.LO
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.LONG_ARR;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.OBJ_ARR;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.PROPS;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.PROXY;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SERIALIZABLE;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SHORT;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SHORT_ARR;
@@ -160,6 +163,9 @@ class OptimizedClassDescriptor {
     /** Access order field offset. */
     private long accessOrderFieldOff;
 
+    /** Proxy interfaces. */
+    private Class<?>[] proxyIntfs;
+
     /**
      * Creates descriptor for class.
      *
@@ -329,6 +335,11 @@ class OptimizedClassDescriptor {
 
                 isCls = true;
             }
+            else if (Proxy.class.isAssignableFrom(cls)) {
+                type = PROXY;
+
+                proxyIntfs = cls.getInterfaces();
+            }
             else {
                 Class<?> c = cls;
 
@@ -558,6 +569,13 @@ class OptimizedClassDescriptor {
     }
 
     /**
+     * @return {@code True} if descriptor is for {@link Proxy}.
+     */
+    boolean isProxy() {
+        return type == PROXY;
+    }
+
+    /**
      * Replaces object.
      *
      * @param obj Object.
@@ -738,6 +756,23 @@ class OptimizedClassDescriptor {
 
                 break;
 
+            case PROXY:
+                out.writeInt(proxyIntfs.length);
+
+                for (Class<?> intf : proxyIntfs) {
+                    OptimizedClassDescriptor intfDesc = classDescriptor(clsMap, intf, ctx, mapper);
+
+                    intfDesc.writeTypeData(out);
+                }
+
+                InvocationHandler ih = Proxy.getInvocationHandler(obj);
+
+                assert ih != null;
+
+                out.writeObject(ih);
+
+                break;
+
             case ENUM:
                 writeTypeData(out);
 
@@ -1017,4 +1052,4 @@ class OptimizedClassDescriptor {
             return fields.get(i);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index fa6f962..923f385 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -136,6 +136,9 @@ class OptimizedMarshallerUtils {
     static final byte CLS = 28;
 
     /** */
+    static final byte PROXY = 29;
+
+    /** */
     static final byte ENUM = 100;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
index 988a777..95a301b 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
@@ -25,8 +25,10 @@ import java.io.ObjectInputValidation;
 import java.io.ObjectStreamClass;
 import java.lang.reflect.Array;
 import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
@@ -77,6 +79,7 @@ import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.LO
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.NULL;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.OBJ_ARR;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.PROPS;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.PROXY;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SERIALIZABLE;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SHORT;
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.SHORT_ARR;
@@ -297,6 +300,16 @@ class OptimizedObjectInputStream extends ObjectInputStream {
             case CLS:
                 return readClass();
 
+            case PROXY:
+                Class<?>[] intfs = new Class<?>[readInt()];
+
+                for (int i = 0; i < intfs.length; i++)
+                    intfs[i] = readClass();
+
+                InvocationHandler ih = (InvocationHandler)readObject();
+
+                return Proxy.newProxyInstance(clsLdr != null ? clsLdr : U.gridClassLoader(), intfs, ih);
+
             case ENUM:
             case EXTERNALIZABLE:
             case SERIALIZABLE:
@@ -1215,4 +1228,4 @@ class OptimizedObjectInputStream extends ObjectInputStream {
             return objs[fieldInfo.getIndex(name)] != null ? (T)objs[fieldInfo.getIndex(name)] : dflt;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
index d884564..96cbbcd 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
@@ -227,7 +227,7 @@ class OptimizedObjectOutputStream extends ObjectOutputStream {
 
                 int handle = -1;
 
-                if (!desc.isPrimitive() && !desc.isEnum() && !desc.isClass())
+                if (!desc.isPrimitive() && !desc.isEnum() && !desc.isClass() && !desc.isProxy())
                     handle = handles.lookup(obj);
 
                 if (obj0 != obj) {
@@ -895,4 +895,4 @@ class OptimizedObjectOutputStream extends ObjectOutputStream {
             objs[i] = F.t(info.type(), val);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/test/java/org/apache/ignite/marshaller/DynamicProxySerializationMultiJvmSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/DynamicProxySerializationMultiJvmSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/DynamicProxySerializationMultiJvmSelfTest.java
new file mode 100644
index 0000000..d22aeac
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/DynamicProxySerializationMultiJvmSelfTest.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.marshaller;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.concurrent.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Multi-JVM test for dynamic proxy serialization.
+ */
+public class DynamicProxySerializationMultiJvmSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static Callable<Marshaller> marshFactory;
+
+    /** {@inheritDoc} */
+    @Override protected boolean isMultiJvm() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(marshFactory.call());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOptimized() throws Exception {
+        marshFactory = new Callable<Marshaller>() {
+            @Override public Marshaller call() throws Exception {
+                return new OptimizedMarshaller(false);
+            }
+        };
+
+        doTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBinary() throws Exception {
+        marshFactory = new Callable<Marshaller>() {
+            @Override public Marshaller call() throws Exception {
+                return new BinaryMarshaller();
+            }
+        };
+
+        doTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTest() throws Exception {
+        try {
+            Ignite ignite = startGrids(2);
+
+            MyProxy p = (MyProxy)Proxy.newProxyInstance(getClass().getClassLoader(),
+                new Class[] { MyProxy.class }, new InvocationHandler() {
+                    @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+                        if ("value".equals(method.getName()))
+                            return 42;
+
+                        throw new IllegalStateException();
+                    }
+                });
+
+            int val = ignite.compute(ignite.cluster().forRemotes()).call(new MyCallable(p));
+
+            assertEquals(42, val);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     */
+    private static class MyCallable implements IgniteCallable<Integer> {
+        /** */
+        private final MyProxy p;
+
+        /**
+         * @param p Proxy.
+         */
+        public MyCallable(MyProxy p) {
+            this.p = p;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            return p.value();
+        }
+    }
+
+    /**
+     */
+    private static interface MyProxy {
+        /**
+         * @return Value.
+         */
+        public int value();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index 0597eda..7d1a37d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.testframework.junits.multijvm;
 
-import com.thoughtworks.xstream.XStream;
 import java.io.BufferedOutputStream;
 import java.io.BufferedReader;
 import java.io.File;
@@ -29,6 +28,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
+import com.thoughtworks.xstream.XStream;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.Ignition;
@@ -100,12 +100,14 @@ public class IgniteNodeRunner {
         String fileName = IGNITE_CONFIGURATION_FILE + cfg.getNodeId();
 
         try(OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) {
-            cfg.setMBeanServer(null);
-            cfg.setMarshaller(null);
-            cfg.setDiscoverySpi(null);
-            cfg.setGridLogger(null);
+            IgniteConfiguration cfg0 = new IgniteConfiguration(cfg);
+
+            cfg0.setMBeanServer(null);
+            cfg0.setMarshaller(null);
+            cfg0.setDiscoverySpi(null);
+            cfg0.setGridLogger(null);
 
-            new XStream().toXML(cfg, out);
+            new XStream().toXML(cfg0, out);
         }
 
         return fileName;
@@ -176,4 +178,4 @@ public class IgniteNodeRunner {
 
         return res;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index fed42e1..a2e0d5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -19,7 +19,6 @@ package org.apache.ignite.testframework.junits.multijvm;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -31,10 +30,10 @@ import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteAtomicReference;
 import org.apache.ignite.IgniteAtomicSequence;
 import org.apache.ignite.IgniteAtomicStamped;
+import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteCountDownLatch;
-import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.IgniteException;
@@ -42,9 +41,9 @@ import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
+import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteServices;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteTransactions;
@@ -76,6 +75,7 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.IgnitePlugin;
 import org.apache.ignite.plugin.PluginNotFoundException;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -121,14 +121,17 @@ public class IgniteProcessProxy implements IgniteEx {
 
         String cfgFileName = IgniteNodeRunner.storeToFile(cfg.setNodeId(id));
 
-        List<String> jvmArgs = U.jvmArgs();
-
         Collection<String> filteredJvmArgs = new ArrayList<>();
 
-        for (String arg : jvmArgs) {
-            if(arg.startsWith("-Xmx") || arg.startsWith("-Xms") ||
+        Marshaller marsh = cfg.getMarshaller();
+
+        if (marsh != null)
+            filteredJvmArgs.add("-D" + IgniteTestResources.MARSH_CLASS_NAME + "=" + marsh.getClass().getName());
+
+        for (String arg : U.jvmArgs()) {
+            if (arg.startsWith("-Xmx") || arg.startsWith("-Xms") ||
                 arg.startsWith("-cp") || arg.startsWith("-classpath") ||
-                arg.startsWith("-D" + IgniteTestResources.MARSH_CLASS_NAME))
+                (marsh != null && arg.startsWith("-D" + IgniteTestResources.MARSH_CLASS_NAME)))
                 filteredJvmArgs.add(arg);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index dece258..c904ef4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesT
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
 import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.DynamicProxySerializationMultiJvmSelfTest;
 import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest;
 import org.apache.ignite.messaging.GridMessagingSelfTest;
 import org.apache.ignite.messaging.IgniteMessagingWithClientTest;
@@ -116,6 +117,8 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
 
+        GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e88cc676/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
index cbb87fa..e0c06dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.util.IgniteUtilsSelfTest;
 import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest;
 import org.apache.ignite.internal.util.nio.GridNioSelfTest;
 import org.apache.ignite.internal.util.nio.GridNioSslSelfTest;
+import org.apache.ignite.marshaller.DynamicProxySerializationMultiJvmSelfTest;
 import org.apache.ignite.marshaller.jdk.GridJdkMarshallerSelfTest;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshallerEnumSelfTest;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshallerNodeFailoverTest;
@@ -84,6 +85,7 @@ public class IgniteBinaryBasicTestSuite extends TestSuite {
         ignoredTests.add(GridMessagingSelfTest.class);
         ignoredTests.add(GridVersionSelfTest.class);
         ignoredTests.add(GridDeploymentMessageCountSelfTest.class);
+        ignoredTests.add(DynamicProxySerializationMultiJvmSelfTest.class);
 
         // TODO: check and delete if pass.
         ignoredTests.add(IgniteDaemonNodeMarshallerCacheTest.class);


[03/43] ignite git commit: Failing tests hotfix

Posted by vk...@apache.org.
Failing tests hotfix


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

Branch: refs/heads/ignite-2249
Commit: 3602d46573ee98ce440fbcc4457f38252aa8351b
Parents: afd3bc1
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Feb 5 12:06:42 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Feb 5 12:06:42 2016 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheAbstractQuerySelfTest.java           | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3602d465/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index 1507543..8ef3f9b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -386,7 +386,15 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
 
         assertEquals(1, res.getValue().intValue());
 
-        U.sleep(1020);
+        U.sleep(800); // Less than minimal amount of time that must pass before a cache entry is considered expired.
+
+        qry =  cache.query(new SqlQuery<Integer, Integer>(Integer.class, "1=1")).getAll();
+
+        res = F.first(qry);
+
+        assertEquals(1, res.getValue().intValue());
+
+        U.sleep(1200); // No expiry guarantee here. Test should be refactored in case of fails.
 
         qry = cache.query(new SqlQuery<Integer, Integer>(Integer.class, "1=1")).getAll();
 


[16/43] ignite git commit: IGNITE-1187: Need to improve project setup in Eclipse

Posted by vk...@apache.org.
IGNITE-1187: Need to improve project setup in Eclipse


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

Branch: refs/heads/ignite-2249
Commit: e18dfdafca3c01ba48958562e1d07c68e71c9302
Parents: a383f2e
Author: Edouard Chevalier <ed...@techmydata.net>
Authored: Mon Feb 8 14:09:03 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Feb 8 14:09:03 2016 +0300

----------------------------------------------------------------------
 .gitignore                   |  3 ++
 modules/yardstick/.gitignore |  2 +
 parent/pom.xml               | 92 ++++++++++++++++++++++++++++++++++++++-
 3 files changed, 96 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e18dfdaf/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index e4e061c..4073a58 100644
--- a/.gitignore
+++ b/.gitignore
@@ -29,3 +29,6 @@ git-patch-prop-local.sh
 **/cpp/**/vs/Win32/
 **/dotnet/**/obj/
 /modules/platforms/cpp/doc/
+.settings
+.classpath
+.project

http://git-wip-us.apache.org/repos/asf/ignite/blob/e18dfdaf/modules/yardstick/.gitignore
----------------------------------------------------------------------
diff --git a/modules/yardstick/.gitignore b/modules/yardstick/.gitignore
new file mode 100644
index 0000000..a2c5e4a
--- /dev/null
+++ b/modules/yardstick/.gitignore
@@ -0,0 +1,2 @@
+/bin
+/libs

http://git-wip-us.apache.org/repos/asf/ignite/blob/e18dfdaf/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 437a30b..e189807 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -513,7 +513,7 @@
                             <Embed-Directory>lib</Embed-Directory>
                             <Embed-Transitive>${osgi.embed.transitive}</Embed-Transitive>
                             <_failok>${osgi.fail.ok}</_failok>
-                            <_invalidfilenames />
+                            <_invalidfilenames/>
                         </instructions>
                     </configuration>
                     <executions>
@@ -1013,5 +1013,95 @@
                 </plugins>
             </build>
         </profile>
+        <profile>
+            <id>m2e</id>
+            <!--This profile is activated when eclipse interacts with maven (using m2e).-->
+            <activation>
+                <property>
+                    <name>m2e.version</name>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <!--eclipse do not support duplicated package-info.java, in both src and test.-->
+                        <artifactId>maven-compiler-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>default-testCompile</id>
+                                <phase>test-compile</phase>
+                                <configuration>
+                                    <testExcludes>
+                                        <exclude>**/package-info.java</exclude>
+                                    </testExcludes>
+                                </configuration>
+                                <goals>
+                                    <goal>testCompile</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+                <pluginManagement>
+                    <plugins>
+                        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+                        <plugin>
+                            <groupId>org.eclipse.m2e</groupId>
+                            <artifactId>lifecycle-mapping</artifactId>
+                            <version>1.0.0</version>
+                            <configuration>
+                                <lifecycleMappingMetadata>
+                                    <pluginExecutions>
+                                        <pluginExecution>
+                                            <pluginExecutionFilter>
+                                                <groupId>org.apache.maven.plugins</groupId>
+                                                <artifactId>maven-antrun-plugin</artifactId>
+                                                <versionRange>[1.7,)</versionRange>
+                                                <goals>
+                                                    <goal>run</goal>
+                                                    <goal>properties-augmentation</goal>
+                                                    <goal>licenses-file-rename</goal>
+                                                </goals>
+                                            </pluginExecutionFilter>
+                                            <action>
+                                                <ignore></ignore>
+                                            </action>
+                                        </pluginExecution>
+                                        <pluginExecution>
+                                            <pluginExecutionFilter>
+                                                <groupId>org.codehaus.mojo</groupId>
+                                                <artifactId>flatten-maven-plugin</artifactId>
+                                                <versionRange>[1.0.0-beta-3,)</versionRange>
+                                                <goals>
+                                                    <goal>flatten</goal>
+                                                </goals>
+                                            </pluginExecutionFilter>
+                                            <action>
+                                                <ignore></ignore>
+                                            </action>
+                                        </pluginExecution>
+                                        <pluginExecution>
+                                            <pluginExecutionFilter>
+                                                <groupId>org.codehaus.mojo</groupId>
+                                                <artifactId>exec-maven-plugin</artifactId>
+                                                <versionRange>[1.3.2,)</versionRange>
+                                                <goals>
+                                                    <goal>java</goal>
+                                                    <goal>default</goal>
+                                                </goals>
+                                            </pluginExecutionFilter>
+                                            <action>
+                                                <ignore></ignore>
+                                            </action>
+                                        </pluginExecution>
+                                    </pluginExecutions>
+                                </lifecycleMappingMetadata>
+                            </configuration>
+                        </plugin>
+                    </plugins>
+                </pluginManagement>
+            </build>
+        </profile>
+
     </profiles>
 </project>


[32/43] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vk...@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/fa3706f8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/fa3706f8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/fa3706f8

Branch: refs/heads/ignite-2249
Commit: fa3706f866b42418abaa54c5f4e73f2dedb166b0
Parents: c3aa137 4c05fc0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 10 15:01:00 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 15:01:00 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheLazyEntry.java        |   3 +
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheMapEntry.java     | 118 +++-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  79 ++-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  85 ++-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  38 +-
 .../cache/query/GridCacheQueryManager.java      |  30 +-
 .../continuous/CacheContinuousQueryHandler.java |   3 +-
 .../CacheContinuousQueryListener.java           |   2 +-
 .../continuous/CacheContinuousQueryManager.java | 120 +++-
 .../continuous/GridContinuousProcessor.java     |  16 +-
 .../IgniteCacheEntryListenerAbstractTest.java   | 454 ++++++++----
 ...cheEntryListenerAtomicOffheapTieredTest.java |  32 +
 ...cheEntryListenerAtomicOffheapValuesTest.java |  32 +
 ...teCacheEntryListenerTxOffheapTieredTest.java |  32 +
 ...teCacheEntryListenerTxOffheapValuesTest.java |  32 +
 .../cache/IgniteCacheEntryListenerTxTest.java   |   1 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |  10 +
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |  33 +
 ...tinuousQueryFailoverTxOffheapTieredTest.java |  32 +
 ...acheContinuousQueryRandomOperationsTest.java | 684 +++++++++++++++++++
 ...ridCacheContinuousQueryAbstractSelfTest.java |  19 +-
 ...eContinuousQueryAtomicOffheapTieredTest.java |  32 +
 ...eContinuousQueryAtomicOffheapValuesTest.java |  32 +
 ...CacheContinuousQueryTxOffheapTieredTest.java |  32 +
 ...CacheContinuousQueryTxOffheapValuesTest.java |  32 +
 .../junits/common/GridCommonAbstractTest.java   |   2 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   8 +
 .../IgniteCacheQuerySelfTestSuite.java          |  14 +
 .../commands/tasks/VisorTasksCommand.scala      |   4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   4 +
 32 files changed, 1749 insertions(+), 276 deletions(-)
----------------------------------------------------------------------



[15/43] ignite git commit: Merge remote-tracking branch 'apache-main/master' into master-main

Posted by vk...@apache.org.
Merge remote-tracking branch 'apache-main/master' into master-main


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

Branch: refs/heads/ignite-2249
Commit: a383f2e79401ecf5f60526aeec4bd8fa51679013
Parents: bf3a5ea d844e95
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Feb 8 13:43:06 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Feb 8 13:43:06 2016 +0300

----------------------------------------------------------------------
 .../Binary/BinarySelfTest.cs                    |  32 ++++
 .../Impl/Binary/BinaryUtils.cs                  | 166 ++++++++++++++++---
 2 files changed, 172 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[36/43] ignite git commit: Fixed devnotes.txt for yarn and mesos modules.

Posted by vk...@apache.org.
Fixed devnotes.txt for yarn and mesos modules.


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

Branch: refs/heads/ignite-2249
Commit: 14824a19847091d5599f41eeaf52aa2694b7da87
Parents: 5539cba
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Feb 10 15:46:27 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Feb 10 15:46:27 2016 +0300

----------------------------------------------------------------------
 DEVNOTES.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/14824a19/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 3e19243..e920b79 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -110,7 +110,7 @@ cd to ./modules/mesos
 
 mvn clean package
 
-Look for ignite-mesos-<version>-jar-with-dependencies.jar in ./target directory.
+Look for ignite-mesos-<version>.jar in ./target directory.
 
 Ignite Yarn Maven Build Instructions
 ============================================
@@ -118,7 +118,7 @@ cd to ./modules/yarn
 
 mvn clean package
 
-Look for ignite-yarn-<version>-jar-with-dependencies.jar in ./target directory.
+Look for ignite-yarn-<version>.jar in ./target directory.
 
 Run tests
 ==========


[04/43] ignite git commit: IGNITE-2508: Fixed hotspot in GridDhtPartitionTopologyImpl caused by excessive HashSet allocations from "nodes(int, AffinityTopologyVersion)" method.

Posted by vk...@apache.org.
IGNITE-2508: Fixed hotspot in GridDhtPartitionTopologyImpl caused by excessive HashSet allocations from "nodes(int, AffinityTopologyVersion)" method.


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

Branch: refs/heads/ignite-2249
Commit: f3e9ae40d31ebe2bd7c338fa2ec6b9c72fe88c31
Parents: 532b373
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Feb 5 12:23:33 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Feb 5 12:23:33 2016 +0300

----------------------------------------------------------------------
 .../affinity/GridAffinityAssignment.java        | 36 +++++++++++++++++++-
 .../affinity/GridAffinityAssignmentCache.java   |  2 +-
 .../cache/GridCacheAffinityManager.java         | 30 +++++++++++++---
 .../dht/GridDhtPartitionTopologyImpl.java       |  9 +++--
 4 files changed, 67 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f3e9ae40/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
index 810843c..7b2bea3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.affinity;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -31,7 +32,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 /**
  * Cached affinity calculations.
  */
-class GridAffinityAssignment implements Serializable {
+public class GridAffinityAssignment implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -47,6 +48,9 @@ class GridAffinityAssignment implements Serializable {
     /** Map of backup node partitions. */
     private final Map<UUID, Set<Integer>> backup;
 
+    /** Assignment node IDs */
+    private transient volatile List<HashSet<UUID>> assignmentIds;
+
     /**
      * Constructs cached affinity calculations item.
      *
@@ -112,6 +116,36 @@ class GridAffinityAssignment implements Serializable {
     }
 
     /**
+     * Get affinity node IDs for partition.
+     *
+     * @param part Partition.
+     * @return Affinity nodes IDs.
+     */
+    public HashSet<UUID> getIds(int part) {
+        assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" +
+            " [part=" + part + ", partitions=" + assignment.size() + ']';
+
+        List<HashSet<UUID>> assignmentIds0 = assignmentIds;
+
+        if (assignmentIds0 == null) {
+            assignmentIds0 = new ArrayList<>();
+
+            for (List<ClusterNode> assignmentPart : assignment) {
+                HashSet<UUID> partIds = new HashSet<>();
+
+                for (ClusterNode node : assignmentPart)
+                    partIds.add(node.id());
+
+                assignmentIds0.add(partIds);
+            }
+
+            assignmentIds = assignmentIds0;
+        }
+
+        return assignmentIds0.get(part);
+    }
+
+    /**
      * Get primary partitions for specified node ID.
      *
      * @param nodeId Node ID to get primary partitions for.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f3e9ae40/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index d728927..26e4d98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -428,7 +428,7 @@ public class GridAffinityAssignmentCache {
      * @param topVer Topology version.
      * @return Cached affinity.
      */
-    private GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) {
+    public GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) {
         if (topVer.equals(AffinityTopologyVersion.NONE))
             topVer = lastVersion();
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/f3e9ae40/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 375219a..21975da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -29,6 +24,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -36,6 +32,12 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
 /**
  * Cache affinity manager.
  */
@@ -246,6 +248,24 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * Get affinity assignment for the given topology version.
+     *
+     * @param topVer Toplogy version.
+     * @return Affinity affignment.
+     */
+    public GridAffinityAssignment assignment(AffinityTopologyVersion topVer) {
+        if (cctx.isLocal())
+            topVer = new AffinityTopologyVersion(1);
+
+        GridAffinityAssignmentCache aff0 = aff;
+
+        if (aff0 == null)
+            throw new IgniteException(FAILED_TO_FIND_CACHE_ERR_MSG + cctx.name());
+
+        return aff0.cachedAffinity(topVer);
+    }
+
+    /**
      * @param key Key to check.
      * @param topVer Topology version.
      * @return Primary node for given key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f3e9ae40/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 2ab8a12..d6fc8f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -36,6 +36,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
@@ -693,7 +694,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public Collection<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
-        Collection<ClusterNode> affNodes = cctx.affinity().nodes(p, topVer);
+        GridAffinityAssignment affAssignment = cctx.affinity().assignment(topVer);
+
+        Collection<ClusterNode> affNodes = affAssignment.get(p);
 
         lock.readLock().lock();
 
@@ -708,9 +711,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             Collection<UUID> nodeIds = part2node.get(p);
 
             if (!F.isEmpty(nodeIds)) {
-                Collection<UUID> affIds = new HashSet<>(F.viewReadOnly(affNodes, F.node2id()));
-
                 for (UUID nodeId : nodeIds) {
+                    HashSet<UUID> affIds = affAssignment.getIds(p);
+
                     if (!affIds.contains(nodeId) && hasState(p, nodeId, OWNING, MOVING, RENTING)) {
                         ClusterNode n = cctx.discovery().node(nodeId);
 


[33/43] ignite git commit: Added test.

Posted by vk...@apache.org.
Added test.


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

Branch: refs/heads/ignite-2249
Commit: 16927abbb1ed1a6c3b47831562d263dd38f495d1
Parents: fa3706f
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 10 15:05:56 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 10 15:06:31 2016 +0300

----------------------------------------------------------------------
 .../cache/CacheQueryBuildValueTest.java         | 144 +++++++++++++++++++
 1 file changed, 144 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16927abb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryBuildValueTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryBuildValueTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryBuildValueTest.java
new file mode 100644
index 0000000..cb574bb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryBuildValueTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class CacheQueryBuildValueTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(null);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        QueryEntity entity = new QueryEntity();
+        entity.setKeyType(Integer.class.getName());
+        entity.setValueType(TestBuilderValue.class.getName());
+
+        ArrayList<QueryIndex> idxs = new ArrayList<>();
+
+        QueryIndex idx = new QueryIndex("iVal");
+        idxs.add(idx);
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("iVal", Integer.class.getName());
+
+        entity.setFields(fields);
+
+        entity.setIndexes(idxs);
+
+        ccfg.setQueryEntities(Collections.singleton(entity));
+
+        cfg.setCacheConfiguration(ccfg);
+
+        BinaryConfiguration binaryCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration typeCfg = new BinaryTypeConfiguration();
+        typeCfg.setTypeName(TestBuilderValue.class.getName());
+
+        binaryCfg.setTypeConfigurations(Collections.singletonList(typeCfg));
+
+        cfg.setBinaryConfiguration(binaryCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBuilderAndQuery() throws Exception {
+        Ignite node = ignite(0);
+
+        final IgniteCache<Object, Object> cache = node.cache(null);
+
+        IgniteBinary binary = node.binary();
+
+        BinaryObjectBuilder builder = binary.builder(TestBuilderValue.class.getName());
+
+        cache.put(0, builder.build());
+
+        builder.setField("iVal", 1);
+
+        cache.put(1, builder.build());
+
+        List<Cache.Entry<Object, Object>> entries =
+            cache.query(new SqlQuery<>(TestBuilderValue.class, "true")).getAll();
+
+        assertEquals(2, entries.size());
+    }
+
+    /**
+     *
+     */
+    static class TestBuilderValue implements Serializable {
+        /** */
+        private int iVal;
+
+        /**
+         * @param iVal Integer value.
+         */
+        public TestBuilderValue(int iVal) {
+            this.iVal = iVal;
+        }
+    }
+}


[27/43] ignite git commit: IGNITE-1507 Correct help text for 'tasks' command in visorcmd.

Posted by vk...@apache.org.
IGNITE-1507 Correct help text for 'tasks' command in visorcmd.


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

Branch: refs/heads/ignite-2249
Commit: 0b47d5cbc6a6bf87e2ab121a0e50d795e3f9ed17
Parents: c67e2ea
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Wed Feb 10 17:13:50 2016 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Feb 10 17:13:50 2016 +0700

----------------------------------------------------------------------
 .../apache/ignite/visor/commands/tasks/VisorTasksCommand.scala   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b47d5cb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
index e158506..660c5f1 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
@@ -250,7 +250,7 @@ private case class VisorTask(
  * |       | of Event Storage SPI that is responsible for temporary storage of generated   |
  * |       | events on each node can also affect the functionality of this command.        |
  * |       |                                                                               |
- * |       | By default - all events are enabled and Ignite stores last 10,000 local       |
+ * |       | By default - all events are disabled and Ignite stores last 10,000 local       |
  * |       | events on each node. Both of these defaults can be changed in configuration.  |
  * +---------------------------------------------------------------------------------------+
  * }}}
@@ -1341,7 +1341,7 @@ object VisorTasksCommand {
             "of Event Storage SPI that is responsible for temporary storage of generated",
             "events on each node can also affect the functionality of this command.",
             " ",
-            "By default - all events are enabled and Ignite stores last 10,000 local",
+            "By default - all events are disabled and Ignite stores last 10,000 local",
             "events on each node. Both of these defaults can be changed in configuration."
         ),
         spec = List(


[42/43] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-2249

Posted by vk...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-2249


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

Branch: refs/heads/ignite-2249
Commit: 6ff7391a81cc7ea402484f3603cda89b8e49fc6f
Parents: 68d6130 35b0e6b
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Feb 11 15:01:47 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Feb 11 15:01:47 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   3 +
 DEVNOTES.txt                                    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  12 +-
 .../ignite/binary/BinaryBasicNameMapper.java    |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  14 +-
 .../internal/binary/BinaryClassDescriptor.java  |  22 +
 .../internal/binary/BinaryReaderExImpl.java     |   8 +-
 .../ignite/internal/binary/BinaryUtils.java     |  48 +-
 .../ignite/internal/binary/BinaryWriteMode.java |   3 +
 .../internal/binary/BinaryWriterExImpl.java     |  34 +
 .../internal/binary/GridBinaryMarshaller.java   |   3 +
 .../affinity/GridAffinityAssignment.java        |  36 +-
 .../affinity/GridAffinityAssignmentCache.java   |   2 +-
 .../cache/CacheEvictableEntryImpl.java          |   4 +-
 .../processors/cache/CacheLazyEntry.java        |   3 +
 .../processors/cache/GridCacheAdapter.java      |  47 +-
 .../cache/GridCacheAffinityManager.java         |  30 +-
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheMapEntry.java     | 118 +++-
 .../processors/cache/GridCacheMvccManager.java  |  42 +-
 .../processors/cache/GridCachePreloader.java    |   6 +
 .../cache/GridCachePreloaderAdapter.java        |   5 +
 .../processors/cache/GridCacheUtils.java        |  21 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   6 +-
 .../dht/GridClientPartitionTopology.java        |   5 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  72 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |  13 +-
 .../cache/distributed/dht/GridDhtGetFuture.java | 176 +++--
 .../distributed/dht/GridDhtGetSingleFuture.java | 476 +++++++++++++
 .../distributed/dht/GridDhtLocalPartition.java  |  76 ++-
 .../distributed/dht/GridDhtPartitionState.java  |   2 +-
 .../dht/GridDhtPartitionTopology.java           |   5 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  18 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  40 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  81 ++-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  85 ++-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  38 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  40 +-
 .../dht/preloader/GridDhtPreloader.java         |  16 +
 ...arOptimisticSerializableTxPrepareFuture.java |   4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   2 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |  67 +-
 .../cache/query/GridCacheQueryManager.java      |  30 +-
 .../continuous/CacheContinuousQueryHandler.java |   3 +-
 .../CacheContinuousQueryListener.java           |   2 +-
 .../continuous/CacheContinuousQueryManager.java | 120 +++-
 .../IgniteCacheObjectProcessorImpl.java         |   2 +-
 .../continuous/GridContinuousProcessor.java     |  16 +-
 .../GridCacheAtomicReferenceImpl.java           |  68 +-
 .../internal/processors/igfs/IgfsProcessor.java |  14 +
 .../platform/PlatformNoopProcessor.java         |  10 +
 .../processors/platform/PlatformProcessor.java  |  20 +
 .../platform/PlatformProcessorImpl.java         |  18 +
 .../callback/PlatformCallbackUtils.java         |   1 -
 .../datastructures/PlatformAtomicReference.java | 141 ++++
 .../datastructures/PlatformAtomicSequence.java  | 122 ++++
 .../util/future/GridCompoundFuture.java         |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 143 +++-
 .../util/nio/GridSelectorNioSessionImpl.java    |   2 +-
 .../util/nio/SelectedSelectionKeySet.java       | 132 ++++
 .../org/apache/ignite/lang/IgniteBiTuple.java   |   6 +-
 .../optimized/OptimizedClassDescriptor.java     |  37 +-
 .../optimized/OptimizedMarshallerUtils.java     |   3 +
 .../optimized/OptimizedObjectInputStream.java   |  15 +-
 .../optimized/OptimizedObjectOutputStream.java  |   4 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |  98 +++
 .../GridCacheOffHeapValuesEvictionSelfTest.java | 171 +++++
 .../IgniteCacheEntryListenerAbstractTest.java   | 454 ++++++++----
 ...cheEntryListenerAtomicOffheapTieredTest.java |  32 +
 ...cheEntryListenerAtomicOffheapValuesTest.java |  32 +
 ...niteCacheEntryListenerExpiredEventsTest.java | 202 ++++++
 ...teCacheEntryListenerTxOffheapTieredTest.java |  32 +
 ...teCacheEntryListenerTxOffheapValuesTest.java |  32 +
 .../cache/IgniteCacheEntryListenerTxTest.java   |   1 +
 .../IgniteTxPreloadAbstractTest.java            |   2 +-
 .../near/GridCacheNearReadersSelfTest.java      |  19 +-
 .../GridCacheReplicatedPreloadSelfTest.java     | 120 ++--
 ...ContinuousQueryFailoverAbstractSelfTest.java |  10 +
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |  33 +
 ...tinuousQueryFailoverTxOffheapTieredTest.java |  32 +
 ...acheContinuousQueryRandomOperationsTest.java | 684 +++++++++++++++++++
 ...ridCacheContinuousQueryAbstractSelfTest.java |  19 +-
 ...eContinuousQueryAtomicOffheapTieredTest.java |  32 +
 ...eContinuousQueryAtomicOffheapValuesTest.java |  32 +
 ...CacheContinuousQueryTxOffheapTieredTest.java |  32 +
 ...CacheContinuousQueryTxOffheapValuesTest.java |  32 +
 .../igfs/IgfsProcessorValidationSelfTest.java   |  27 +
 .../apache/ignite/lang/GridTupleSelfTest.java   |  42 +-
 ...namicProxySerializationMultiJvmSelfTest.java | 131 ++++
 .../junits/common/GridCommonAbstractTest.java   |   2 +-
 .../junits/multijvm/IgniteNodeRunner.java       |  16 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  19 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   5 +
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |  10 +
 .../hadoop/fs/BasicHadoopFileSystemFactory.java |  22 +-
 .../fs/CachingHadoopFileSystemFactory.java      |   2 +-
 .../fs/KerberosHadoopFileSystemFactory.java     | 217 ++++++
 ...KerberosHadoopFileSystemFactorySelfTest.java | 121 ++++
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 .../cache/CacheQueryBuildValueTest.java         | 144 ++++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  10 +-
 .../IgniteCacheQuerySelfTestSuite.java          |  14 +
 .../osgi-karaf/src/main/resources/features.xml  |  12 +-
 .../cpp/common/include/ignite/common/exports.h  |  15 +
 .../cpp/common/include/ignite/common/java.h     |  32 +
 .../platforms/cpp/common/project/vs/module.def  |  15 +-
 modules/platforms/cpp/common/src/exports.cpp    |  52 ++
 modules/platforms/cpp/common/src/java.cpp       | 181 +++++
 modules/platforms/cpp/core-test/Makefile.am     |   1 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   1 +
 .../project/vs/core-test.vcxproj.filters        |   3 +
 .../platforms/cpp/core-test/src/cache_test.cpp  |  12 +
 .../cpp/core-test/src/interop_memory_test.cpp   |  95 +++
 .../include/ignite/impl/ignite_environment.h    |  19 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   2 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  30 +-
 .../Apache.Ignite.Core.Tests.csproj             |   6 +
 .../Binary/BinarySelfTest.cs                    |  32 +
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../DataStructures/AtomicReferenceTest.cs       | 239 +++++++
 .../DataStructures/AtomicSequenceTest.cs        | 131 ++++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |   1 +
 .../IgniteConfigurationSectionTest.cs           |  69 ++
 .../IgniteConfigurationSerializerTest.cs        | 554 +++++++++++++++
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |  54 ++
 .../Apache.Ignite.Core.csproj                   |  15 +
 .../Binary/IBinarySerializer.cs                 |   6 +-
 .../Cache/CachePartialUpdateException.cs        |   1 -
 .../Cache/Configuration/QueryEntity.cs          |   4 +-
 .../Cache/Store/CacheStoreAdapter.cs            |   1 -
 .../Compute/ComputeTaskAdapter.cs               |   1 -
 .../Compute/ComputeTaskSplitAdapter.cs          |   1 -
 .../DataStructures/IAtomicReference.cs          |  64 ++
 .../DataStructures/IAtomicSequence.cs           |  69 ++
 .../Multicast/TcpDiscoveryMulticastIpFinder.cs  |   4 +-
 .../Tcp/Static/TcpDiscoveryStaticIpFinder.cs    |   6 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |   1 -
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |  28 +
 .../IgniteConfigurationSection.cs               |  80 +++
 .../IgniteConfigurationSection.xsd              | 281 ++++++++
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  39 ++
 .../Impl/Binary/BinaryUtils.cs                  | 166 ++++-
 .../Impl/Common/BooleanLowerCaseConverter.cs    |  60 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  14 +-
 .../Impl/Common/FutureType.cs                   |  18 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  | 407 +++++++++++
 .../Impl/Common/TypeStringConverter.cs          | 115 ++++
 .../Closure/ComputeAbstractClosureTask.cs       |   1 -
 .../Impl/DataStructures/AtomicReference.cs      |  92 +++
 .../Impl/DataStructures/AtomicSequence.cs       |  90 +++
 .../Impl/Events/EventTypeConverter.cs           | 133 ++++
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   8 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  50 ++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  12 +
 .../Impl/Memory/PlatformMemoryStream.cs         | 320 +++++++--
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  37 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  79 +++
 .../commands/tasks/VisorTasksCommand.scala      |   4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   4 +
 modules/yardstick/.gitignore                    |   2 +
 parent/pom.xml                                  |  93 ++-
 163 files changed, 8520 insertions(+), 840 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ff7391a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ff7391a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ff7391a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------


[41/43] ignite git commit: IGNITE-2555

Posted by vk...@apache.org.
 IGNITE-2555


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

Branch: refs/heads/ignite-2249
Commit: 35b0e6bf149bb86a3eefefcbc657c822e25681f3
Parents: 877be93
Author: ruskim <ru...@gmail.com>
Authored: Thu Feb 11 18:53:50 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Feb 11 18:53:50 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 14 ++-
 .../internal/GridNodeMetricsLogSelfTest.java    | 98 ++++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |  2 +
 3 files changed, 113 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35b0e6bf/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index e3017ff..5d8daf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -989,6 +989,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                             double avgCpuLoadPct = m.getAverageCpuLoad() * 100;
                             double gcPct = m.getCurrentGcCpuLoad() * 100;
 
+                            //Heap params
                             long heapUsed = m.getHeapMemoryUsed();
                             long heapMax = m.getHeapMemoryMaximum();
 
@@ -997,6 +998,15 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
                             double freeHeapPct = heapMax > 0 ? ((double)((heapMax - heapUsed) * 100)) / heapMax : -1;
 
+                            //Non heap params
+                            long nonHeapUsed = m.getNonHeapMemoryUsed();
+                            long nonHeapMax = m.getNonHeapMemoryMaximum();
+
+                            long nonHeapUsedInMBytes = nonHeapUsed / 1024 / 1024;
+                            long nonHeapCommInMBytes = m.getNonHeapMemoryCommitted() / 1024 / 1024;
+
+                            double freeNonHeapPct = nonHeapMax > 0 ? ((double)((nonHeapMax - nonHeapUsed) * 100)) / nonHeapMax : -1;
+
                             int hosts = 0;
                             int nodes = 0;
                             int cpus = 0;
@@ -1046,12 +1056,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
                             String msg = NL +
                                 "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
-                                "    ^-- Node [id=" + id + ", name=" + name() + "]" + NL +
+                                "    ^-- Node [id=" + id + ", name=" + name() + ", uptime=" + getUpTimeFormatted() + "]" + NL +
                                 "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
                                 "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
                                 dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
                                 "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
                                 dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
+                                "    ^-- Non heap [used=" + dblFmt.format(nonHeapUsedInMBytes) + "MB, free=" +
+                                dblFmt.format(freeNonHeapPct) + "%, comm=" + dblFmt.format(nonHeapCommInMBytes) + "MB]" + NL +
                                 "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
                                 pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
                                 "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b0e6bf/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
new file mode 100644
index 0000000..fe5922e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+
+import java.io.StringWriter;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonTest;
+import org.apache.log4j.Layout;
+import org.apache.log4j.Logger;
+import org.apache.log4j.SimpleLayout;
+import org.apache.log4j.WriterAppender;
+
+/**
+ * Check logging local node metrics
+ */
+@SuppressWarnings({"ProhibitedExceptionDeclared"})
+@GridCommonTest(group = "Kernal")
+public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
+    /** */
+
+    public GridNodeMetricsLogSelfTest() {
+        super(false);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"unchecked"})
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMetricsLogFrequency(1000);
+
+        return cfg;
+    }
+
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeMetricsLog() throws Exception {
+        // Log to string, to check log content
+        Layout layout = new SimpleLayout();
+
+        StringWriter strWr = new StringWriter();
+
+        WriterAppender app = new WriterAppender(layout, strWr);
+
+        Logger.getRootLogger().addAppender(app);
+
+        Ignite g1 = startGrid(1);
+
+        IgniteCache<Integer, String> cache1 = g1.createCache("TestCache1");
+
+        cache1.put(1, "one");
+
+        Ignite g2 = startGrid(2);
+
+        IgniteCache<Integer, String> cache2 = g2.createCache("TestCache2");
+
+        cache2.put(2, "two");
+
+        Thread.sleep(10000);
+
+        //Check that nodes are alie
+        assert cache1.get(1).equals("one");
+        assert cache2.get(2).equals("two");
+
+        String fullLog = strWr.toString();
+
+        Logger.getRootLogger().removeAppender(app);
+
+        assert fullLog.contains("Metrics for local node");
+        assert fullLog.contains("uptime=");
+        assert fullLog.contains("Non heap");
+        assert fullLog.contains("Outbound messages queue");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b0e6bf/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index c904ef4..3903910 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
 import org.apache.ignite.internal.GridLifecycleAwareSelfTest;
 import org.apache.ignite.internal.GridLifecycleBeanSelfTest;
+import org.apache.ignite.internal.GridNodeMetricsLogSelfTest;
 import org.apache.ignite.internal.GridProjectionForCachesSelfTest;
 import org.apache.ignite.internal.GridReduceSelfTest;
 import org.apache.ignite.internal.GridReleaseTypeSelfTest;
@@ -114,6 +115,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, IgniteDaemonNodeMarshallerCacheTest.class, ignoredTests);
         suite.addTestSuite(IgniteMarshallerCacheConcurrentReadWriteTest.class);
+        suite.addTestSuite(GridNodeMetricsLogSelfTest.class);
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
 


[37/43] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vk...@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/d08a7790
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d08a7790
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d08a7790

Branch: refs/heads/ignite-2249
Commit: d08a779083634b2d332961c8ae098e2e2cf041ed
Parents: 008c8cd 14824a1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Feb 10 15:46:36 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 15:46:36 2016 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   4 +-
 ...niteCacheEntryListenerExpiredEventsTest.java | 202 +++++++++++++++++++
 .../cache/CacheQueryBuildValueTest.java         | 144 +++++++++++++
 3 files changed, 348 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[24/43] ignite git commit: IGNITE-2380: Added ability to start Ignite using configuration from app.config. This closes #417.

Posted by vk...@apache.org.
IGNITE-2380: Added ability to start Ignite using configuration from app.config. This closes #417.


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

Branch: refs/heads/ignite-2249
Commit: a4d8a049138a4d4bc86a59ccb273f7e214baf694
Parents: f7c1296
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Feb 9 14:54:20 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Feb 9 14:54:20 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |   4 +
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |   1 +
 .../IgniteConfigurationSectionTest.cs           |  69 +++
 .../IgniteConfigurationSerializerTest.cs        | 554 +++++++++++++++++++
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |  54 ++
 .../Apache.Ignite.Core.csproj                   |  11 +
 .../Binary/IBinarySerializer.cs                 |   6 +-
 .../IgniteConfigurationSection.cs               |  80 +++
 .../IgniteConfigurationSection.xsd              | 281 ++++++++++
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |  39 ++
 .../Impl/Common/BooleanLowerCaseConverter.cs    |  60 ++
 .../Common/IgniteConfigurationXmlSerializer.cs  | 410 ++++++++++++++
 .../Impl/Common/TypeStringConverter.cs          | 115 ++++
 .../Impl/Events/EventTypeConverter.cs           | 133 +++++
 14 files changed, 1814 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index f5e98c5..fb14ed5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -55,6 +55,7 @@
       <HintPath>..\libs\nunit.framework.dll</HintPath>
     </Reference>
     <Reference Include="System" />
+    <Reference Include="System.configuration" />
     <Reference Include="System.Core" />
     <Reference Include="System.Runtime.Serialization" />
     <Reference Include="System.XML" />
@@ -117,6 +118,8 @@
     <Compile Include="ExceptionsTest.cs" />
     <Compile Include="ExecutableTest.cs" />
     <Compile Include="FutureTest.cs" />
+    <Compile Include="IgniteConfigurationSectionTest.cs" />
+    <Compile Include="IgniteConfigurationSerializerTest.cs" />
     <Compile Include="IgniteConfigurationTest.cs" />
     <Compile Include="IgniteTestBase.cs" />
     <Compile Include="LifecycleTest.cs" />
@@ -251,6 +254,7 @@
   <ItemGroup>
     <None Include="Apache.Ignite.Core.Tests.nunit" />
     <None Include="Apache.Ignite.Core.Tests.snk" />
+    <None Include="app.config" />
   </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <PropertyGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
index f18be8c..bc1f08f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/FutureTest.cs
@@ -67,6 +67,7 @@ namespace Apache.Ignite.Core.Tests
         [TestFixtureTearDown]
         public void TestFixtureTearDown()
         {
+            Ignition.StopAll(true);
             TestUtils.KillProcesses();
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
new file mode 100644
index 0000000..29aea90
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests
+{
+    using System;
+    using System.Configuration;
+    using System.Linq;
+    using Apache.Ignite.Core.Impl.Common;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests <see cref="IgniteConfigurationSection"/>.
+    /// </summary>
+    public class IgniteConfigurationSectionTest
+    {
+        /// <summary>
+        /// Tests the read.
+        /// </summary>
+        [Test]
+        public void TestRead()
+        {
+            var section = (IgniteConfigurationSection) ConfigurationManager.GetSection("igniteConfiguration");
+
+            Assert.AreEqual("myGrid1", section.IgniteConfiguration.GridName);
+            Assert.AreEqual("cacheName", section.IgniteConfiguration.CacheConfiguration.Single().Name);
+        }
+
+        /// <summary>
+        /// Tests the ignite start.
+        /// </summary>
+        [Test]
+        public void TestIgniteStart()
+        {
+            Environment.SetEnvironmentVariable(Classpath.EnvIgniteNativeTestClasspath, "true");
+
+            using (var ignite = Ignition.StartFromApplicationConfiguration("igniteConfiguration"))
+            {
+                Assert.AreEqual("myGrid1", ignite.Name);
+                Assert.IsNotNull(ignite.GetCache<int, int>("cacheName"));
+            }
+
+            using (var ignite = Ignition.StartFromApplicationConfiguration("igniteConfiguration2"))
+            {
+                Assert.AreEqual("myGrid2", ignite.Name);
+                Assert.IsNotNull(ignite.GetCache<int, int>("cacheName2"));
+            }
+
+            using (var ignite = Ignition.StartFromApplicationConfiguration())
+            {
+                Assert.IsTrue(ignite.Name.StartsWith("myGrid"));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
new file mode 100644
index 0000000..d8c52ee
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -0,0 +1,554 @@
+/*
+ * 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.
+ */
+
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+// ReSharper disable MemberCanBePrivate.Global
+namespace Apache.Ignite.Core.Tests
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Globalization;
+    using System.IO;
+    using System.Linq;
+    using System.Text;
+    using System.Threading;
+    using System.Xml;
+    using System.Xml.Schema;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Discovery.Tcp.Multicast;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Tests.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests <see cref="IgniteConfiguration"/> serialization.
+    /// </summary>
+    public class IgniteConfigurationSerializerTest
+    {
+        /// <summary>
+        /// Tests the predefined XML.
+        /// </summary>
+        [Test]
+        public void TestPredefinedXml()
+        {
+            var xml = @"<igniteConfig workDirectory='c:' JvmMaxMemoryMb='1024' MetricsLogFrequency='0:0:10'>
+                            <localhost>127.1.1.1</localhost>
+                            <binaryConfiguration>
+                                <defaultNameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' bar='testBar' />
+                                <types>
+                                    <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
+                                </types>
+                            </binaryConfiguration>
+                            <discoverySpi type='TcpDiscoverySpi' joinTimeout='0:1:0'>
+                                <ipFinder type='TcpDiscoveryMulticastIpFinder' addressRequestAttempts='7' />
+                            </discoverySpi>
+                            <jvmOptions><string>-Xms1g</string><string>-Xmx4g</string></jvmOptions>
+                            <lifecycleBeans>
+                                <iLifecycleBean type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+LifecycleBean, Apache.Ignite.Core.Tests' foo='15' />
+                            </lifecycleBeans>
+                            <cacheConfiguration>
+                                <cacheConfiguration cacheMode='Replicated'>
+                                    <queryEntities>    
+                                        <queryEntity keyType='System.Int32' valueType='System.String'>    
+                                            <fields>
+                                                <queryField name='length' fieldType='System.Int32' />
+                                            </fields>
+                                            <aliases>
+                                                <queryAlias fullName='somefield.field' alias='shortField' />
+                                            </aliases>
+                                            <indexes>
+                                                <queryIndex name='idx' indexType='Geospatial'>
+                                                    <fields>
+                                                        <queryIndexField name='indexFld' isDescending='true' />
+                                                    </fields>
+                                                </queryIndex>
+                                            </indexes>
+                                        </queryEntity>
+                                    </queryEntities>
+                                </cacheConfiguration>
+                                <cacheConfiguration name='secondCache' />
+                            </cacheConfiguration>
+                            <includedEventTypes>
+                                <int>42</int>
+                                <int>TaskFailed</int>
+                                <int>JobFinished</int>
+                            </includedEventTypes>
+                        </igniteConfig>";
+            var reader = XmlReader.Create(new StringReader(xml));
+
+            var cfg = IgniteConfigurationXmlSerializer.Deserialize(reader);
+
+            Assert.AreEqual("c:", cfg.WorkDirectory);
+            Assert.AreEqual("127.1.1.1", cfg.Localhost);
+            Assert.AreEqual(1024, cfg.JvmMaxMemoryMb);
+            Assert.AreEqual(TimeSpan.FromSeconds(10), cfg.MetricsLogFrequency);
+            Assert.AreEqual(TimeSpan.FromMinutes(1), ((TcpDiscoverySpi)cfg.DiscoverySpi).JoinTimeout);
+            Assert.AreEqual(7,
+                ((TcpDiscoveryMulticastIpFinder) ((TcpDiscoverySpi) cfg.DiscoverySpi).IpFinder).AddressRequestAttempts);
+            Assert.AreEqual(new[] { "-Xms1g", "-Xmx4g" }, cfg.JvmOptions);
+            Assert.AreEqual(15, ((LifecycleBean) cfg.LifecycleBeans.Single()).Foo);
+            Assert.AreEqual("testBar", ((NameMapper) cfg.BinaryConfiguration.DefaultNameMapper).Bar);
+            Assert.AreEqual(
+                "Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests",
+                cfg.BinaryConfiguration.Types.Single());
+            Assert.AreEqual(new[] {42, EventType.TaskFailed, EventType.JobFinished}, cfg.IncludedEventTypes);
+
+            Assert.AreEqual("secondCache", cfg.CacheConfiguration.Last().Name);
+
+            var cacheCfg = cfg.CacheConfiguration.First();
+
+            Assert.AreEqual(CacheMode.Replicated, cacheCfg.CacheMode);
+
+            var queryEntity = cacheCfg.QueryEntities.Single();
+            Assert.AreEqual(typeof(int), queryEntity.KeyType);
+            Assert.AreEqual(typeof(string), queryEntity.ValueType);
+            Assert.AreEqual("length", queryEntity.Fields.Single().Name);
+            Assert.AreEqual(typeof(int), queryEntity.Fields.Single().FieldType);
+            Assert.AreEqual("somefield.field", queryEntity.Aliases.Single().FullName);
+            Assert.AreEqual("shortField", queryEntity.Aliases.Single().Alias);
+            Assert.AreEqual(QueryIndexType.Geospatial, queryEntity.Indexes.Single().IndexType);
+            Assert.AreEqual("indexFld", queryEntity.Indexes.Single().Fields.Single().Name);
+            Assert.AreEqual(true, queryEntity.Indexes.Single().Fields.Single().IsDescending);
+        }
+
+        /// <summary>
+        /// Tests the serialize deserialize.
+        /// </summary>
+        [Test]
+        public void TestSerializeDeserialize()
+        {
+            // Test custom
+            CheckSerializeDeserialize(GetTestConfig());
+
+            // Test custom with different culture to make sure numbers are serialized properly
+            RunWithCustomCulture(() => CheckSerializeDeserialize(GetTestConfig()));
+            
+            // Test default
+            CheckSerializeDeserialize(new IgniteConfiguration());
+        }
+
+        /// <summary>
+        /// Tests the schema validation.
+        /// </summary>
+        [Test]
+        public void TestSchemaValidation()
+        {
+            CheckSchemaValidation();
+
+            RunWithCustomCulture(CheckSchemaValidation);
+
+            // Check invalid xml
+            const string invalidXml =
+                @"<igniteConfiguration xmlns='http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection'>
+                    <binaryConfiguration /><binaryConfiguration />
+                  </igniteConfiguration>";
+
+            Assert.Throws<XmlSchemaValidationException>(() => CheckSchemaValidation(invalidXml));
+        }
+
+        /// <summary>
+        /// Checks the schema validation.
+        /// </summary>
+        private static void CheckSchemaValidation()
+        {
+            var sb = new StringBuilder();
+
+            using (var xmlWriter = XmlWriter.Create(sb))
+            {
+                IgniteConfigurationXmlSerializer.Serialize(GetTestConfig(), xmlWriter, "igniteConfiguration");
+            }
+
+            CheckSchemaValidation(sb.ToString());
+        }
+
+        /// <summary>
+        /// Checks the schema validation.
+        /// </summary>
+        /// <param name="xml">The XML.</param>
+        private static void CheckSchemaValidation(string xml)
+        {
+            var document = new XmlDocument();
+
+            document.Schemas.Add("http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection", 
+                XmlReader.Create("IgniteConfigurationSection.xsd"));
+
+            document.Load(new StringReader(xml));
+
+            document.Validate(null);
+        }
+
+        /// <summary>
+        /// Checks the serialize deserialize.
+        /// </summary>
+        /// <param name="cfg">The config.</param>
+        private static void CheckSerializeDeserialize(IgniteConfiguration cfg)
+        {
+            var resCfg = SerializeDeserialize(cfg);
+
+            AssertReflectionEqual(cfg, resCfg);
+        }
+
+        /// <summary>
+        /// Serializes and deserializes a config.
+        /// </summary>
+        private static IgniteConfiguration SerializeDeserialize(IgniteConfiguration cfg)
+        {
+            var sb = new StringBuilder();
+
+            using (var xmlWriter = XmlWriter.Create(sb))
+            {
+                IgniteConfigurationXmlSerializer.Serialize(cfg, xmlWriter, "igniteConfig");
+            }
+
+            var xml = sb.ToString();
+
+            using (var xmlReader = XmlReader.Create(new StringReader(xml)))
+            {
+                xmlReader.MoveToContent();
+                return IgniteConfigurationXmlSerializer.Deserialize(xmlReader);
+            }
+        }
+
+        /// <summary>
+        /// Asserts equality with reflection.
+        /// </summary>
+        private static void AssertReflectionEqual(object x, object y)
+        {
+            var type = x.GetType();
+
+            Assert.AreEqual(type, y.GetType());
+
+            if (type.IsValueType || type == typeof (string) || type.IsSubclassOf(typeof (Type)))
+            {
+                Assert.AreEqual(x, y);
+                return;
+            }
+
+            var props = type.GetProperties();
+
+            foreach (var propInfo in props)
+            {
+                var propType = propInfo.PropertyType;
+
+                var xVal = propInfo.GetValue(x, null);
+                var yVal = propInfo.GetValue(y, null);
+
+                if (xVal == null || yVal == null)
+                {
+                    Assert.IsNull(xVal);
+                    Assert.IsNull(yVal);
+                }
+                else if (propType != typeof(string) && propType.IsGenericType 
+                    && propType.GetGenericTypeDefinition() == typeof (ICollection<>))
+                {
+                    var xCol = ((IEnumerable) xVal).OfType<object>().ToList();
+                    var yCol = ((IEnumerable) yVal).OfType<object>().ToList();
+
+                    Assert.AreEqual(xCol.Count, yCol.Count);
+
+                    for (int i = 0; i < xCol.Count; i++)
+                        AssertReflectionEqual(xCol[i], yCol[i]);
+                }
+                else
+                {
+                    AssertReflectionEqual(xVal, yVal);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Gets the test configuration.
+        /// </summary>
+        private static IgniteConfiguration GetTestConfig()
+        {
+            return new IgniteConfiguration
+            {
+                GridName = "gridName",
+                JvmOptions = new[] {"1", "2"},
+                Localhost = "localhost11",
+                JvmClasspath = "classpath",
+                Assemblies = new[] {"asm1", "asm2", "asm3"},
+                BinaryConfiguration = new BinaryConfiguration
+                {
+                    TypeConfigurations = new[]
+                    {
+                        new BinaryTypeConfiguration
+                        {
+                            IsEnum = true,
+                            KeepDeserialized = true,
+                            AffinityKeyFieldName = "affKeyFieldName",
+                            TypeName = "typeName",
+                            IdMapper = new IdMapper(),
+                            NameMapper = new NameMapper(),
+                            Serializer = new TestSerializer()
+                        }
+                    },
+                    DefaultIdMapper = new IdMapper(),
+                    DefaultKeepDeserialized = true,
+                    DefaultNameMapper = new NameMapper(),
+                    DefaultSerializer = new TestSerializer()
+                },
+                CacheConfiguration = new[]
+                {
+                    new CacheConfiguration("cacheName")
+                    {
+                        AtomicWriteOrderMode = CacheAtomicWriteOrderMode.Primary,
+                        AtomicityMode = CacheAtomicityMode.Transactional,
+                        Backups = 15,
+                        CacheMode = CacheMode.Partitioned,
+                        CacheStoreFactory = new TestCacheStoreFactory(),
+                        CopyOnRead = true,
+                        EagerTtl = true,
+                        EnableSwap = true,
+                        EvictSynchronized = true,
+                        EvictSynchronizedConcurrencyLevel = 13,
+                        EvictSynchronizedKeyBufferSize = 14,
+                        EvictSynchronizedTimeout = TimeSpan.FromMinutes(3),
+                        Invalidate = true,
+                        KeepBinaryInStore = true,
+                        LoadPreviousValue = true,
+                        LockTimeout = TimeSpan.FromSeconds(56),
+                        LongQueryWarningTimeout = TimeSpan.FromSeconds(99),
+                        MaxConcurrentAsyncOperations = 24,
+                        MaxEvictionOverflowRatio = 5.6F,
+                        MemoryMode = CacheMemoryMode.OffheapValues,
+                        OffHeapMaxMemory = 567,
+                        QueryEntities = new[]
+                        {
+                            new QueryEntity
+                            {
+                                Fields = new[]
+                                {
+                                    new QueryField("field", typeof (int))
+                                },
+                                Indexes = new[]
+                                {
+                                    new QueryIndex("field") { IndexType = QueryIndexType.FullText }
+                                },
+                                Aliases = new[]
+                                {
+                                    new QueryAlias("field.field", "fld")
+                                },
+                                KeyType = typeof (string),
+                                ValueType = typeof (long)
+                            },
+                        },
+                        ReadFromBackup = true,
+                        RebalanceBatchSize = 33,
+                        RebalanceDelay = TimeSpan.MaxValue,
+                        RebalanceMode = CacheRebalanceMode.Sync,
+                        RebalanceThrottle = TimeSpan.FromHours(44),
+                        RebalanceTimeout = TimeSpan.FromMinutes(8),
+                        SqlEscapeAll = true,
+                        SqlOnheapRowCacheSize = 679,
+                        StartSize = 1023,
+                        WriteBehindBatchSize = 45,
+                        WriteBehindEnabled = true,
+                        WriteBehindFlushFrequency = TimeSpan.FromSeconds(5),
+                        WriteBehindFlushSize = 66,
+                        WriteBehindFlushThreadCount = 2,
+                        WriteSynchronizationMode = CacheWriteSynchronizationMode.FullAsync
+                    }
+                },
+                ClientMode = true,
+                DiscoverySpi = new TcpDiscoverySpi
+                {
+                    NetworkTimeout = TimeSpan.FromSeconds(1),
+                    SocketTimeout = TimeSpan.FromSeconds(2),
+                    AckTimeout = TimeSpan.FromSeconds(3),
+                    JoinTimeout = TimeSpan.FromSeconds(4),
+                    MaxAckTimeout = TimeSpan.FromSeconds(5),
+                    IpFinder = new TcpDiscoveryMulticastIpFinder
+                    {
+                        TimeToLive = 110,
+                        MulticastGroup = "multicastGroup",
+                        AddressRequestAttempts = 10,
+                        MulticastPort = 987,
+                        ResponseTimeout = TimeSpan.FromDays(1),
+                        LocalAddress = "127.0.0.2",
+                        Endpoints = new[] {"", "abc"}
+                    }
+                },
+                IgniteHome = "igniteHome",
+                IncludedEventTypes = EventType.CacheQueryAll,
+                JvmDllPath = @"c:\jvm",
+                JvmInitialMemoryMb = 1024,
+                JvmMaxMemoryMb = 2048,
+                LifecycleBeans = new[] {new LifecycleBean(), new LifecycleBean() },
+                MetricsExpireTime = TimeSpan.FromSeconds(15),
+                MetricsHistorySize = 45,
+                MetricsLogFrequency = TimeSpan.FromDays(2),
+                MetricsUpdateFrequency = TimeSpan.MinValue,
+                NetworkSendRetryCount = 7,
+                NetworkSendRetryDelay = TimeSpan.FromSeconds(98),
+                NetworkTimeout = TimeSpan.FromMinutes(4),
+                SuppressWarnings = true,
+                WorkDirectory = @"c:\work"
+            };
+        }
+
+        /// <summary>
+        /// Runs the with custom culture.
+        /// </summary>
+        /// <param name="action">The action.</param>
+        private static void RunWithCustomCulture(Action action)
+        {
+            RunWithCulture(action, CultureInfo.InvariantCulture);
+            RunWithCulture(action, CultureInfo.GetCultureInfo("ru-RU"));
+        }
+
+        /// <summary>
+        /// Runs the with culture.
+        /// </summary>
+        /// <param name="action">The action.</param>
+        /// <param name="cultureInfo">The culture information.</param>
+        private static void RunWithCulture(Action action, CultureInfo cultureInfo)
+        {
+            var oldCulture = Thread.CurrentThread.CurrentCulture;
+
+            try
+            {
+                Thread.CurrentThread.CurrentCulture = cultureInfo;
+
+                action();
+            }
+            finally
+            {
+                Thread.CurrentThread.CurrentCulture = oldCulture;
+            }
+        }
+
+        /// <summary>
+        /// Test bean.
+        /// </summary>
+        public class LifecycleBean : ILifecycleBean
+        {
+            /// <summary>
+            /// Gets or sets the foo.
+            /// </summary>
+            /// <value>
+            /// The foo.
+            /// </value>
+            public int Foo { get; set; }
+
+            /// <summary>
+            /// This method is called when lifecycle event occurs.
+            /// </summary>
+            /// <param name="evt">Lifecycle event.</param>
+            public void OnLifecycleEvent(LifecycleEventType evt)
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Test mapper.
+        /// </summary>
+        public class NameMapper : IBinaryNameMapper
+        {
+            /// <summary>
+            /// Gets or sets the bar.
+            /// </summary>
+            /// <value>
+            /// The bar.
+            /// </value>
+            public string Bar { get; set; }
+
+            /// <summary>
+            /// Gets the type name.
+            /// </summary>
+            /// <param name="name">The name.</param>
+            /// <returns>
+            /// Type name.
+            /// </returns>
+            public string GetTypeName(string name)
+            {
+                return name;
+            }
+
+            /// <summary>
+            /// Gets the field name.
+            /// </summary>
+            /// <param name="name">The name.</param>
+            /// <returns>
+            /// Field name.
+            /// </returns>
+            public string GetFieldName(string name)
+            {
+                return name;
+            }
+        }
+
+        /// <summary>
+        /// Serializer.
+        /// </summary>
+        public class TestSerializer : IBinarySerializer
+        {
+            /// <summary>
+            /// Write portalbe object.
+            /// </summary>
+            /// <param name="obj">Object.</param>
+            /// <param name="writer">Poratble writer.</param>
+            public void WriteBinary(object obj, IBinaryWriter writer)
+            {
+                // No-op.
+            }
+
+            /// <summary>
+            /// Read binary object.
+            /// </summary>
+            /// <param name="obj">Instantiated empty object.</param>
+            /// <param name="reader">Poratble reader.</param>
+            public void ReadBinary(object obj, IBinaryReader reader)
+            {
+                // No-op.
+            }
+        }
+
+        /// <summary>
+        /// Test class.
+        /// </summary>
+        public class FooClass
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Test factory.
+        /// </summary>
+        public class TestCacheStoreFactory : IFactory<ICacheStore>
+        {
+            /// <summary>
+            /// Creates an instance of the cache store.
+            /// </summary>
+            /// <returns>
+            /// New instance of the cache store.
+            /// </returns>
+            public ICacheStore CreateInstance()
+            {
+                return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
new file mode 100644
index 0000000..c290c83
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
@@ -0,0 +1,54 @@
+<?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.
+-->
+
+<configuration>
+    <configSections>
+        <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+    </configSections>
+
+    <igniteConfiguration xmlns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection" gridName="myGrid1">
+        <discoverySpi type="TcpDiscoverySpi">
+            <ipFinder type="TcpDiscoveryStaticIpFinder">
+                <endpoints>
+                    <string>127.0.0.1:47500..47501</string>
+                </endpoints>
+            </ipFinder>
+        </discoverySpi>
+        
+        <cacheConfiguration>
+            <cacheConfiguration name="cacheName" />
+        </cacheConfiguration>
+    </igniteConfiguration>
+
+    <igniteConfiguration2 gridName="myGrid2">
+        <discoverySpi type="TcpDiscoverySpi">
+            <ipFinder type="TcpDiscoveryStaticIpFinder">
+                <endpoints>
+                    <string>127.0.0.1:47500..47501</string>
+                </endpoints>
+            </ipFinder>
+        </discoverySpi>
+
+        <cacheConfiguration>
+            <cacheConfiguration name="cacheName2" />
+        </cacheConfiguration>
+    </igniteConfiguration2>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index e2efd0a..d0ef352 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -60,7 +60,9 @@
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />
+    <Reference Include="System.configuration" />
     <Reference Include="System.Core" />
+    <Reference Include="System.Xml" />
   </ItemGroup>
   <ItemGroup>
     <Compile Include="Binary\Package-Info.cs" />
@@ -183,6 +185,7 @@
     <Compile Include="Events\TaskEvent.cs" />
     <Compile Include="GlobalSuppressions.cs" />
     <Compile Include="IgniteConfiguration.cs" />
+    <Compile Include="IgniteConfigurationSection.cs" />
     <Compile Include="Ignition.cs" />
     <Compile Include="IIgnite.cs" />
     <Compile Include="Impl\Binary\BinaryEnum.cs" />
@@ -222,11 +225,13 @@
     <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
     <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />
     <Compile Include="Impl\Collections\ReadOnlyDictionary.cs" />
+    <Compile Include="Impl\Common\BooleanLowerCaseConverter.cs" />
     <Compile Include="Impl\Common\CancelledTask.cs" />
     <Compile Include="Impl\Common\Classpath.cs" />
     <Compile Include="Impl\Common\CopyOnWriteConcurrentDictionary.cs" />
     <Compile Include="Impl\Common\DelegateConverter.cs" />
     <Compile Include="Impl\Common\DelegateTypeDescriptor.cs" />
+    <Compile Include="Impl\Events\EventTypeConverter.cs" />
     <Compile Include="Impl\Common\Fnv1Hash.cs" />
     <Compile Include="Impl\Common\Future.cs" />
     <Compile Include="Impl\Common\FutureConverter.cs" />
@@ -234,10 +239,12 @@
     <Compile Include="Impl\Common\IgniteArgumentCheck.cs" />
     <Compile Include="Impl\Common\IFutureConverter.cs" />
     <Compile Include="Impl\Common\IFutureInternal.cs" />
+    <Compile Include="Impl\Common\IgniteConfigurationXmlSerializer.cs" />
     <Compile Include="Impl\Common\IgniteHome.cs" />
     <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
     <Compile Include="Impl\Common\ResizeableArray.cs" />
     <Compile Include="Impl\Common\TypeCaster.cs" />
+    <Compile Include="Impl\Common\TypeStringConverter.cs" />
     <Compile Include="Impl\Compute\Closure\ComputeAbstractClosureTask.cs" />
     <Compile Include="Impl\Compute\Closure\ComputeActionJob.cs" />
     <Compile Include="Impl\Compute\Closure\ComputeFuncJob.cs" />
@@ -434,6 +441,10 @@
   <ItemGroup>
     <None Include="Apache.Ignite.Core.ruleset" />
     <None Include="Apache.Ignite.Core.snk" />
+    <None Include="IgniteConfigurationSection.xsd">
+      <SubType>Designer</SubType>
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </None>
   </ItemGroup>
   <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinarySerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinarySerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinarySerializer.cs
index 23dc811..3f924eb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinarySerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/IBinarySerializer.cs
@@ -23,17 +23,17 @@ namespace Apache.Ignite.Core.Binary
     public interface IBinarySerializer
     {
         /// <summary>
-        /// Write portalbe object.
+        /// Write binary object.
         /// </summary>
         /// <param name="obj">Object.</param>
-        /// <param name="writer">Poratble writer.</param>
+        /// <param name="writer">Binary writer.</param>
         void WriteBinary(object obj, IBinaryWriter writer);
 
         /// <summary>
         /// Read binary object.
         /// </summary>
         /// <param name="obj">Instantiated empty object.</param>
-        /// <param name="reader">Poratble reader.</param>
+        /// <param name="reader">Binary reader.</param>
         void ReadBinary(object obj, IBinaryReader reader);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.cs
new file mode 100644
index 0000000..51b963e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.cs
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core
+{
+    using System.Configuration;
+    using System.Text;
+    using System.Xml;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Ignite configuration section for app.config and web.config files.
+    /// </summary>
+    public class IgniteConfigurationSection : ConfigurationSection
+    {
+        /// <summary>
+        /// Gets or sets the ignite configuration.
+        /// </summary>
+        /// <value>
+        /// The ignite configuration.
+        /// </value>
+        public IgniteConfiguration IgniteConfiguration { get; private set; }
+
+        /// <summary>
+        /// Reads XML from the configuration file.
+        /// </summary>
+        /// <param name="reader">The reader object, which reads from the configuration file.</param>
+        protected override void DeserializeSection(XmlReader reader)
+        {
+            IgniteArgumentCheck.NotNull(reader, "reader");
+
+            IgniteConfiguration = IgniteConfigurationXmlSerializer.Deserialize(reader);
+        }
+
+        /// <summary>
+        /// Creates an XML string containing an unmerged view of the <see cref="ConfigurationSection" /> 
+        /// object as a single section to write to a file.
+        /// </summary>
+        /// <param name="parentElement">The <see cref="ConfigurationElement" /> 
+        /// instance to use as the parent when performing the un-merge.</param>
+        /// <param name="name">The name of the section to create.</param>
+        /// <param name="saveMode">The <see cref="ConfigurationSaveMode" /> instance 
+        /// to use when writing to a string.</param>
+        /// <returns>
+        /// An XML string containing an unmerged view of the <see cref="ConfigurationSection" /> object.
+        /// </returns>
+        protected override string SerializeSection(ConfigurationElement parentElement, string name, 
+            ConfigurationSaveMode saveMode)
+        {
+            IgniteArgumentCheck.NotNull(parentElement, "parentElement");
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            if (IgniteConfiguration == null)
+                return string.Format("<{0} />", name);
+
+            var sb = new StringBuilder();
+
+            using (var xmlWriter = XmlWriter.Create(sb))
+            {
+                IgniteConfigurationXmlSerializer.Serialize(IgniteConfiguration, xmlWriter, name);
+
+                return sb.ToString();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
new file mode 100644
index 0000000..5181217
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -0,0 +1,281 @@
+<?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.
+-->
+
+<xs:schema id="IgniteConfigurationSection"
+    targetNamespace="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"
+    elementFormDefault="qualified"
+    xmlns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"
+    xmlns:mstns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"
+    xmlns:xs="http://www.w3.org/2001/XMLSchema">
+
+    <xs:element name="igniteConfiguration">
+        <xs:complexType>
+            <xs:all>
+                <xs:element name="binaryConfiguration" minOccurs="0">
+                    <xs:complexType>
+                        <xs:all>
+                            <xs:element name="typeConfigurations" minOccurs="0">
+                                <xs:complexType>
+                                    <xs:sequence>
+                                        <xs:element name="binaryTypeConfiguration">
+                                            <xs:complexType>
+                                                <xs:all>
+                                                    <xs:element name="nameMapper" minOccurs="0">
+                                                        <xs:complexType>
+                                                            <xs:attribute name="type" type="xs:string" use="required" />
+                                                        </xs:complexType>
+                                                    </xs:element>
+                                                    <xs:element name="idMapper" minOccurs="0">
+                                                        <xs:complexType>
+                                                            <xs:attribute name="type" type="xs:string" use="required" />
+                                                        </xs:complexType>
+                                                    </xs:element>
+                                                    <xs:element name="serializer" minOccurs="0">
+                                                        <xs:complexType>
+                                                            <xs:attribute name="type" type="xs:string" use="required" />
+                                                        </xs:complexType>
+                                                    </xs:element>
+                                                </xs:all>
+                                                <xs:attribute name="typeName" type="xs:string" />
+                                                <xs:attribute name="affinityKeyFieldName" type="xs:string" />
+                                                <xs:attribute name="keepDeserialized" type="xs:string" />
+                                                <xs:attribute name="isEnum" type="xs:boolean" />
+                                            </xs:complexType>
+                                        </xs:element>
+                                    </xs:sequence>
+                                </xs:complexType>
+                            </xs:element>
+                            <xs:element name="defaultNameMapper" minOccurs="0">
+                                <xs:complexType>
+                                    <xs:attribute name="type" type="xs:string" use="required" />
+                                </xs:complexType>
+                            </xs:element>
+                            <xs:element name="defaultIdMapper" minOccurs="0">
+                                <xs:complexType>
+                                    <xs:attribute name="type" type="xs:string" use="required" />
+                                </xs:complexType>
+                            </xs:element>
+                            <xs:element name="defaultSerializer" minOccurs="0">
+                                <xs:complexType>
+                                    <xs:attribute name="type" type="xs:string" use="required" />
+                                </xs:complexType>
+                            </xs:element>
+                        </xs:all>
+                        <xs:attribute name="defaultKeepDeserialized" type="xs:boolean" />
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="cacheConfiguration" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element name="cacheConfiguration" maxOccurs="unbounded">
+                                <xs:complexType>
+                                    <xs:all>
+                                        <xs:element name="cacheStoreFactory" minOccurs="0">
+                                            <xs:complexType>
+                                                <xs:attribute name="type" type="xs:string" use="required" />
+                                            </xs:complexType>
+                                        </xs:element>
+                                        <xs:element name="queryEntities" minOccurs="0">
+                                            <xs:complexType>
+                                                <xs:sequence>
+                                                    <xs:element name="queryEntity" maxOccurs="unbounded">
+                                                        <xs:complexType>
+                                                            <xs:all>
+                                                                <xs:element name="fields" minOccurs="0">
+                                                                    <xs:complexType>
+                                                                        <xs:sequence>
+                                                                            <xs:element name="queryField" maxOccurs="unbounded">
+                                                                                <xs:complexType>
+                                                                                    <xs:attribute name="name" type="xs:string" use="required" />
+                                                                                    <xs:attribute name="fieldType" type="xs:string" />
+                                                                                    <xs:attribute name="fieldTypeName" type="xs:string" />
+                                                                                </xs:complexType>
+                                                                            </xs:element>
+                                                                        </xs:sequence>
+                                                                    </xs:complexType>
+                                                                </xs:element>
+                                                                <xs:element name="aliases" minOccurs="0">
+                                                                    <xs:complexType>
+                                                                        <xs:sequence>
+                                                                            <xs:element name="queryAlias" maxOccurs="unbounded">
+                                                                                <xs:complexType>
+                                                                                    <xs:attribute name="fullName" type="xs:string" use="required" />
+                                                                                    <xs:attribute name="alias" type="xs:string" use="required" />
+                                                                                </xs:complexType>
+                                                                            </xs:element>
+                                                                        </xs:sequence>
+                                                                    </xs:complexType>
+                                                                </xs:element>
+                                                                <xs:element name="indexes" minOccurs="0">
+                                                                    <xs:complexType>
+                                                                        <xs:sequence>
+                                                                            <xs:element name="queryIndex" maxOccurs="unbounded">
+                                                                                <xs:complexType>
+                                                                                    <xs:sequence>
+                                                                                        <xs:element name="fields" minOccurs="1">
+                                                                                            <xs:complexType>
+                                                                                                <xs:sequence>
+                                                                                                    <xs:element name="queryIndexField" maxOccurs="unbounded">
+                                                                                                        <xs:complexType>
+                                                                                                            <xs:attribute name="name" type="xs:string" use="required" />
+                                                                                                            <xs:attribute name="isDescending" type="xs:boolean" />
+                                                                                                        </xs:complexType>
+                                                                                                    </xs:element>
+                                                                                                </xs:sequence>
+                                                                                            </xs:complexType>
+                                                                                        </xs:element>
+                                                                                    </xs:sequence>
+                                                                                    <xs:attribute name="name" type="xs:string" />
+                                                                                    <xs:attribute name="indexType" type="xs:string" />
+                                                                                </xs:complexType>
+                                                                            </xs:element>
+                                                                        </xs:sequence>
+                                                                    </xs:complexType>
+                                                                </xs:element>
+                                                            </xs:all>
+                                                            <xs:attribute name="keyTypeName" type="xs:string"/>
+                                                            <xs:attribute name="keyType" type="xs:string" />
+                                                            <xs:attribute name="valueTypeName" type="xs:string" />
+                                                            <xs:attribute name="valueType" type="xs:string" />
+                                                        </xs:complexType>
+                                                    </xs:element>
+                                                </xs:sequence>
+                                            </xs:complexType>
+                                        </xs:element>
+                                    </xs:all>
+                                    <xs:attribute name="name" type="xs:string" />
+                                    <xs:attribute name="writeSynchronizationMode" type="xs:string" />
+                                    <xs:attribute name="evictSynchronized" type="xs:boolean" />
+                                    <xs:attribute name="evictSynchronizedKeyBufferSize" type="xs:int" />
+                                    <xs:attribute name="evictSynchronizedConcurrencyLevel" type="xs:int" />
+                                    <xs:attribute name="evictSynchronizedTimeout" type="xs:string" />
+                                    <xs:attribute name="maxEvictionOverflowRatio" type="xs:decimal" />
+                                    <xs:attribute name="startSize" type="xs:int" />
+                                    <xs:attribute name="loadPreviousValue" type="xs:string" />
+                                    <xs:attribute name="atomicityMode" type="xs:string" />
+                                    <xs:attribute name="atomicWriteOrderMode" type="xs:string" />
+                                    <xs:attribute name="backups" type="xs:int" />
+                                    <xs:attribute name="lockTimeout" type="xs:string" />
+                                    <xs:attribute name="invalidate" type="xs:boolean" />
+                                    <xs:attribute name="rebalanceMode" type="xs:string" />
+                                    <xs:attribute name="rebalanceBatchSize" type="xs:int" />
+                                    <xs:attribute name="enableSwap" type="xs:boolean" />
+                                    <xs:attribute name="maxConcurrentAsyncOperations" type="xs:int" />
+                                    <xs:attribute name="writeBehindEnabled" type="xs:boolean" />
+                                    <xs:attribute name="writeBehindFlushSize" type="xs:int" />
+                                    <xs:attribute name="writeBehindFlushThreadCount" type="xs:int" />
+                                    <xs:attribute name="writeBehindBatchSize" type="xs:int" />
+                                    <xs:attribute name="rebalanceTimeout" type="xs:string" />
+                                    <xs:attribute name="rebalanceDelay" type="xs:string" />
+                                    <xs:attribute name="rebalanceThrottle" type="xs:string" />
+                                    <xs:attribute name="offHeapMaxMemory" type="xs:int" />
+                                    <xs:attribute name="memoryMode" type="xs:string" />
+                                    <xs:attribute name="longQueryWarningTimeout" type="xs:string" />
+                                    <xs:attribute name="sqlEscapeAll" type="xs:boolean" />
+                                    <xs:attribute name="sqlOnheapRowCacheSize" type="xs:int" />
+                                </xs:complexType>
+                            </xs:element>
+                        </xs:sequence>
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="jvmOptions" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element maxOccurs="unbounded" name="string" type="xs:string" />
+                        </xs:sequence>
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="assemblies" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element maxOccurs="unbounded" name="string" type="xs:string" />
+                        </xs:sequence>
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="lifecycleBeans" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element maxOccurs="unbounded" name="iLifecycleBean">
+                                <xs:complexType>
+                                    <xs:attribute name="type" type="xs:string" use="required" />
+                                </xs:complexType>
+                            </xs:element>
+                        </xs:sequence>
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="discoverySpi" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element name="ipFinder" minOccurs="0">
+                                <xs:complexType>
+                                    <xs:sequence>
+                                        <xs:element name="endpoints" minOccurs="0">
+                                            <xs:complexType>
+                                                <xs:sequence>
+                                                    <xs:element maxOccurs="unbounded" name="string" type="xs:string" />
+                                                </xs:sequence>
+                                            </xs:complexType>
+                                        </xs:element>
+                                    </xs:sequence>
+                                    <xs:attribute name="type" type="xs:string" use="required" />
+                                    <xs:attribute name="localAddress" type="xs:string" />
+                                    <xs:attribute name="multicastGroup" type="xs:string" />
+                                    <xs:attribute name="multicastPort" type="xs:int" />
+                                    <xs:attribute name="addressRequestAttempts" type="xs:int" />
+                                    <xs:attribute name="responseTimeout" type="xs:string" />
+                                    <xs:attribute name="timeToLive" type="xs:byte" />
+                                </xs:complexType>
+                            </xs:element>
+                        </xs:sequence>
+                        <xs:attribute name="socketTimeout" type="xs:string" />
+                        <xs:attribute name="ackTimeout" type="xs:string" />
+                        <xs:attribute name="maxAckTimeout" type="xs:string" />
+                        <xs:attribute name="networkTimeout" type="xs:string" />
+                        <xs:attribute name="joinTimeout" type="xs:string" />
+                        <xs:attribute name="type" type="xs:string" use="required" />
+                    </xs:complexType>
+                </xs:element>
+                <xs:element name="includedEventTypes" minOccurs="0">
+                    <xs:complexType>
+                        <xs:sequence>
+                            <xs:element maxOccurs="unbounded" name="int" type="xs:string" />
+                        </xs:sequence>
+                    </xs:complexType>
+                </xs:element>
+            </xs:all>
+            <xs:attribute name="gridName" type="xs:string" />
+            <xs:attribute name="jvmDllPath" type="xs:string" />
+            <xs:attribute name="igniteHome" type="xs:string" />
+            <xs:attribute name="jvmClasspath" type="xs:string" />
+            <xs:attribute name="suppressWarnings" type="xs:boolean" />
+            <xs:attribute name="jvmInitialMemoryMb" type="xs:int" />
+            <xs:attribute name="jvmMaxMemoryMb" type="xs:int" />
+            <xs:attribute name="clientMode" type="xs:boolean" />
+            <xs:attribute name="metricsExpireTime" type="xs:string" />
+            <xs:attribute name="metricsHistorySize" type="xs:int" />
+            <xs:attribute name="metricsLogFrequency" type="xs:string" />
+            <xs:attribute name="metricsUpdateFrequency" type="xs:string" />
+            <xs:attribute name="networkSendRetryCount" type="xs:int" />
+            <xs:attribute name="networkSendRetryDelay" type="xs:string" />
+            <xs:attribute name="networkTimeout" type="xs:string" />
+            <xs:attribute name="workDirectory" type="xs:string" />
+            <xs:attribute name="localhost" type="xs:string" />
+        </xs:complexType>
+    </xs:element>    
+</xs:schema>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 0549010..70d7422 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core
 {
     using System;
     using System.Collections.Generic;
+    using System.Configuration;
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Linq;
@@ -116,6 +117,44 @@ namespace Apache.Ignite.Core
         }
 
         /// <summary>
+        /// Reads <see cref="IgniteConfiguration"/> from first <see cref="IgniteConfigurationSection"/> in the 
+        /// application configuration and starts Ignite.
+        /// </summary>
+        /// <returns>Started Ignite.</returns>
+        public static IIgnite StartFromApplicationConfiguration()
+        {
+            var cfg = ConfigurationManager.OpenExeConfiguration(ConfigurationUserLevel.None);
+
+            var section = cfg.Sections.OfType<IgniteConfigurationSection>().FirstOrDefault();
+
+            if (section == null)
+                throw new ConfigurationErrorsException(
+                    string.Format("Could not find {0} in current application configuration",
+                        typeof(IgniteConfigurationSection).Name));
+
+            return Start(section.IgniteConfiguration);
+        }
+
+        /// <summary>
+        /// Reads <see cref="IgniteConfiguration"/> from application configuration 
+        /// <see cref="IgniteConfigurationSection"/> with specified name and starts Ignite.
+        /// </summary>
+        /// <param name="sectionName">Name of the section.</param>
+        /// <returns>Started Ignite.</returns>
+        public static IIgnite StartFromApplicationConfiguration(string sectionName)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(sectionName, "sectionName");
+
+            var section = ConfigurationManager.GetSection(sectionName) as IgniteConfigurationSection;
+
+            if (section == null)
+                throw new ConfigurationErrorsException(string.Format("Could not find {0} with name '{1}'",
+                    typeof(IgniteConfigurationSection).Name, sectionName));
+
+            return Start(section.IgniteConfiguration);
+        }
+
+        /// <summary>
         /// Starts Ignite with given configuration.
         /// </summary>
         /// <returns>Started Ignite.</returns>

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/BooleanLowerCaseConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/BooleanLowerCaseConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/BooleanLowerCaseConverter.cs
new file mode 100644
index 0000000..e8cced2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/BooleanLowerCaseConverter.cs
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.ComponentModel;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Globalization;
+
+    /// <summary>
+    /// Bollean converter that returns lower-case strings, for XML serialization.
+    /// </summary>
+    internal class BooleanLowerCaseConverter : BooleanConverter
+    {
+        /// <summary>
+        /// Default instance.
+        /// </summary>
+        public static readonly BooleanLowerCaseConverter Instance = new BooleanLowerCaseConverter();
+
+        /// <summary>
+        /// Converts the given value object to the specified type, using the specified context and culture information.
+        /// </summary>
+        /// <param name="context">An <see cref="ITypeDescriptorContext" /> that provides a format context.</param>
+        /// <param name="culture">
+        /// A <see cref="CultureInfo" />. If null is passed, the current culture is assumed.
+        /// </param>
+        /// <param name="value">The <see cref="object" /> to convert.</param>
+        /// <param name="destinationType">
+        /// The <see cref="Type" /> to convert the <paramref name="value" /> parameter to.
+        /// </param>
+        /// <returns>
+        /// An <see cref="object" /> that represents the converted value.
+        /// </returns>
+        [SuppressMessage("Microsoft.Globalization", "CA1308:NormalizeStringsToUppercase")]
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "2")]
+        public override object ConvertTo(ITypeDescriptorContext context, CultureInfo culture, object value, 
+            Type destinationType)
+        {
+            if (destinationType == typeof (string))
+                return value.ToString().ToLowerInvariant();
+
+            return base.ConvertTo(context, culture, value, destinationType);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4d8a049/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
new file mode 100644
index 0000000..af25bfa
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
@@ -0,0 +1,410 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.ComponentModel;
+    using System.Configuration;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Linq;
+    using System.Reflection;
+    using System.Xml;
+    using Apache.Ignite.Core.Impl.Events;
+
+    /// <summary>
+    /// Serializes <see cref="IgniteConfiguration"/> to XML.
+    /// </summary>
+    internal static class IgniteConfigurationXmlSerializer
+    {
+        /** Attribute that specifies a type for abstract properties, such as IpFinder. */
+        private const string TypNameAttribute = "type";
+
+        /** Xmlns. */
+        private const string XmlnsAttribute = "xmlns";
+
+        /** Schema. */
+        private const string Schema = "http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection";
+
+        /// <summary>
+        /// Deserializes <see cref="IgniteConfiguration"/> from specified <see cref="XmlReader"/>.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        /// <returns>Resulting <see cref="IgniteConfiguration"/>.</returns>
+        public static IgniteConfiguration Deserialize(XmlReader reader)
+        {
+            IgniteArgumentCheck.NotNull(reader, "reader");
+
+            var cfg = new IgniteConfiguration();
+
+            if (reader.NodeType == XmlNodeType.Element || reader.Read())
+                ReadElement(reader, cfg);
+
+            return cfg;
+        }
+
+        /// <summary>
+        /// Serializes specified <see cref="IgniteConfiguration" /> to <see cref="XmlWriter" />.
+        /// </summary>
+        /// <param name="configuration">The configuration.</param>
+        /// <param name="writer">The writer.</param>
+        /// <param name="rootElementName">Name of the root element.</param>
+        public static void Serialize(IgniteConfiguration configuration, XmlWriter writer, string rootElementName)
+        {
+            IgniteArgumentCheck.NotNull(configuration, "configuration");
+            IgniteArgumentCheck.NotNull(writer, "writer");
+            IgniteArgumentCheck.NotNullOrEmpty(rootElementName, "rootElementName");
+
+            WriteElement(configuration, writer, rootElementName, typeof(IgniteConfiguration));
+        }
+
+        /// <summary>
+        /// Writes new element.
+        /// </summary>
+        private static void WriteElement(object obj, XmlWriter writer, string rootElementName, Type valueType, 
+            PropertyInfo property = null)
+        {
+            if (valueType == typeof(IgniteConfiguration))
+                writer.WriteStartElement(rootElementName, Schema);  // write xmlns for the root element
+            else
+                writer.WriteStartElement(rootElementName);
+
+            if (IsBasicType(valueType))
+                WriteBasicProperty(obj, writer, valueType, property);
+            else if (valueType.IsGenericType && valueType.GetGenericTypeDefinition() == typeof (ICollection<>))
+                WriteCollectionProperty(obj, writer, valueType, property);
+            else
+                WriteComplexProperty(obj, writer, valueType);
+
+            writer.WriteEndElement();
+        }
+
+        /// <summary>
+        /// Writes the property of a basic type (primitives, strings, types).
+        /// </summary>
+        [SuppressMessage("ReSharper", "AssignNullToNotNullAttribute")]
+        private static void WriteBasicProperty(object obj, XmlWriter writer, Type valueType, PropertyInfo property)
+        {
+            var converter = GetConverter(property, valueType);
+
+            var stringValue = converter.ConvertToInvariantString(obj);
+
+            writer.WriteString(stringValue);
+        }
+
+        /// <summary>
+        /// Writes the collection property.
+        /// </summary>
+        private static void WriteCollectionProperty(object obj, XmlWriter writer, Type valueType, PropertyInfo property)
+        {
+            var elementType = valueType.GetGenericArguments().Single();
+
+            var elementTypeName = PropertyNameToXmlName(elementType.Name);
+
+            foreach (var element in (IEnumerable)obj)
+                WriteElement(element, writer, elementTypeName, elementType, property);
+        }
+
+        /// <summary>
+        /// Writes the complex property (nested object).
+        /// </summary>
+        [SuppressMessage("ReSharper", "AssignNullToNotNullAttribute")]
+        private static void WriteComplexProperty(object obj, XmlWriter writer, Type valueType)
+        {
+            var props = GetNonDefaultProperties(obj).ToList();
+
+            // Specify type for interfaces and abstract classes
+            if (valueType.IsAbstract)
+                writer.WriteAttributeString(TypNameAttribute, TypeStringConverter.Convert(obj.GetType()));
+
+            // Write attributes
+            foreach (var prop in props.Where(p => IsBasicType(p.PropertyType)))
+            {
+                var converter = GetConverter(prop, prop.PropertyType);
+                var stringValue = converter.ConvertToInvariantString(prop.GetValue(obj, null));
+                writer.WriteAttributeString(PropertyNameToXmlName(prop.Name), stringValue);
+            }
+
+            // Write elements
+            foreach (var prop in props.Where(p => !IsBasicType(p.PropertyType)))
+                WriteElement(prop.GetValue(obj, null), writer, PropertyNameToXmlName(prop.Name),
+                    prop.PropertyType, prop);
+        }
+
+        /// <summary>
+        /// Reads the element.
+        /// </summary>
+        private static void ReadElement(XmlReader reader, object target)
+        {
+            var targetType = target.GetType();
+
+            // Read attributes
+            while (reader.MoveToNextAttribute())
+            {
+                var name = reader.Name;
+                var val = reader.Value;
+
+                SetProperty(target, name, val);
+            }
+
+            // Read content
+            reader.MoveToElement();
+
+            while (reader.Read())
+            {
+                if (reader.NodeType != XmlNodeType.Element)
+                    continue;
+
+                var name = reader.Name;
+                var prop = GetPropertyOrThrow(name, reader.Value, targetType);
+                var propType = prop.PropertyType;
+
+                if (IsBasicType(propType))
+                {
+                    // Regular property in xmlElement form
+                    SetProperty(target, name, reader.ReadString());
+                }
+                else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (ICollection<>))
+                {
+                    // Collection
+                    ReadCollectionProperty(reader, prop, target);
+                }
+                else
+                {
+                    // Nested object (complex property)
+                    prop.SetValue(target, ReadComplexProperty(reader, propType, prop.Name, targetType), null);
+                }
+            }
+        }
+
+        /// <summary>
+        /// Reads the complex property (nested object).
+        /// </summary>
+        private static object ReadComplexProperty(XmlReader reader, Type propType, string propName, Type targetType)
+        {
+            if (propType.IsAbstract)
+            {
+                var typeName = reader.GetAttribute(TypNameAttribute);
+
+                var derivedTypes = GetConcreteDerivedTypes(propType);
+
+                propType = typeName == null
+                    ? null
+                    : Type.GetType(typeName, false) ?? derivedTypes.FirstOrDefault(x => x.Name == typeName);
+
+                if (propType == null)
+                {
+                    var message = string.Format("'type' attribute is required for '{0}.{1}' property", targetType.Name,
+                        propName);
+
+                    if (typeName != null)
+                    {
+                        message += ", specified type cannot be resolved: " + typeName;
+                    }
+                    else if (derivedTypes.Any())
+                        message += ", possible values are: " + string.Join(", ", derivedTypes.Select(x => x.Name));
+
+                    throw new ConfigurationErrorsException(message);
+                }
+            }
+
+            var nestedVal = Activator.CreateInstance(propType);
+
+            using (var subReader = reader.ReadSubtree())
+            {
+                subReader.Read();  // read first element
+
+                ReadElement(subReader, nestedVal);
+            }
+
+            return nestedVal;
+        }
+
+        /// <summary>
+        /// Reads the collection.
+        /// </summary>
+        private static void ReadCollectionProperty(XmlReader reader, PropertyInfo prop, object target)
+        {
+            var elementType = prop.PropertyType.GetGenericArguments().Single();
+
+            var listType = typeof (List<>).MakeGenericType(elementType);
+
+            var list = (IList) Activator.CreateInstance(listType);
+
+            var converter = IsBasicType(elementType) ? GetConverter(prop, elementType) : null;
+
+            using (var subReader = reader.ReadSubtree())
+            {
+                subReader.Read();  // skip list head
+                while (subReader.Read())
+                {
+                    if (subReader.NodeType != XmlNodeType.Element)
+                        continue;
+
+                    if (subReader.Name != PropertyNameToXmlName(elementType.Name))
+                        throw new ConfigurationErrorsException(
+                            string.Format("Invalid list element in IgniteConfiguration: expected '{0}', but was '{1}'",
+                                PropertyNameToXmlName(elementType.Name), subReader.Name));
+
+                    list.Add(converter != null
+                        ? converter.ConvertFromInvariantString(subReader.ReadString())
+                        : ReadComplexProperty(subReader, elementType, prop.Name, target.GetType()));
+                }
+            }
+
+            prop.SetValue(target, list, null);
+        }
+
+        /// <summary>
+        /// Sets the property.
+        /// </summary>
+        private static void SetProperty(object target, string propName, string propVal)
+        {
+            if (propName == TypNameAttribute || propName == XmlnsAttribute)
+                return;
+
+            var type = target.GetType();
+            var property = GetPropertyOrThrow(propName, propVal, type);
+
+            var converter = GetConverter(property, property.PropertyType);
+
+            var convertedVal = converter.ConvertFromInvariantString(propVal);
+
+            property.SetValue(target, convertedVal, null);
+        }
+
+        /// <summary>
+        /// Gets concrete derived types.
+        /// </summary>
+        private static List<Type> GetConcreteDerivedTypes(Type type)
+        {
+            return type.Assembly.GetTypes().Where(t => t.IsClass && !t.IsAbstract && type.IsAssignableFrom(t)).ToList();
+        }
+
+        /// <summary>
+        /// Gets specified property from a type or throws an exception.
+        /// </summary>
+        private static PropertyInfo GetPropertyOrThrow(string propName, string propVal, Type type)
+        {
+            var property = type.GetProperty(XmlNameToPropertyName(propName));
+
+            if (property == null)
+                throw new ConfigurationErrorsException(
+                    string.Format(
+                        "Invalid IgniteConfiguration attribute '{0}={1}', there is no such property on '{2}'",
+                        propName, propVal, type));
+
+            return property;
+        }
+
+        /// <summary>
+        /// Converts an XML name to CLR name.
+        /// </summary>
+        private static string XmlNameToPropertyName(string name)
+        {
+            Debug.Assert(name.Length > 0);
+
+            if (name == "int")
+                return "Int32";  // allow aliases
+
+            return char.ToUpperInvariant(name[0]) + name.Substring(1);
+        }
+
+        /// <summary>
+        /// Converts a CLR name to XML name.
+        /// </summary>
+        private static string PropertyNameToXmlName(string name)
+        {
+            Debug.Assert(name.Length > 0);
+
+            if (name == "Int32")
+                return "int";  // allow aliases
+
+            return char.ToLowerInvariant(name[0]) + name.Substring(1);
+        }
+
+        /// <summary>
+        /// Determines whether specified type is a basic built-in type.
+        /// </summary>
+        private static bool IsBasicType(Type propertyType)
+        {
+            Debug.Assert(propertyType != null);
+
+            return propertyType.IsValueType || propertyType == typeof(string) || propertyType == typeof(Type);
+        }
+
+        /// <summary>
+        /// Gets converter for a property.
+        /// </summary>
+        private static TypeConverter GetConverter(PropertyInfo property, Type propertyType)
+        {
+            Debug.Assert(property != null);
+            Debug.Assert(propertyType != null);
+
+            if (propertyType.IsEnum)
+                return new GenericEnumConverter(propertyType);
+
+            if (propertyType == typeof (Type))
+                return TypeStringConverter.Instance;
+
+            if (propertyType == typeof(bool))
+                return BooleanLowerCaseConverter.Instance;
+
+            if (property.DeclaringType == typeof (IgniteConfiguration) && property.Name == "IncludedEventTypes")
+                return EventTypeConverter.Instance;
+
+            var converter = TypeDescriptor.GetConverter(propertyType);
+
+            if (converter == null || !converter.CanConvertFrom(typeof(string)) ||
+                !converter.CanConvertTo(typeof(string)))
+                throw new ConfigurationErrorsException("No converter for type " + propertyType);
+
+            return converter;
+        }
+
+        /// <summary>
+        /// Gets properties with non-default value.
+        /// </summary>
+        private static IEnumerable<PropertyInfo> GetNonDefaultProperties(object obj)
+        {
+            Debug.Assert(obj != null);
+
+            return obj.GetType().GetProperties().Where(p => !Equals(p.GetValue(obj, null), GetDefaultValue(p)));
+        }
+
+        /// <summary>
+        /// Gets the default value for a property.
+        /// </summary>
+        private static object GetDefaultValue(PropertyInfo property)
+        {
+            var attr = property.GetCustomAttributes(true).OfType<DefaultValueAttribute>().FirstOrDefault();
+
+            if (attr != null)
+                return attr.Value;
+
+            var propertyType = property.PropertyType;
+
+            if (propertyType.IsValueType)
+                return Activator.CreateInstance(propertyType);
+
+            return null;
+        }
+    }
+}


[18/43] ignite git commit: IGNITE-2264: Minor refactoring to GridDistributedTxMapping mapping handling.

Posted by vk...@apache.org.
IGNITE-2264: Minor refactoring to GridDistributedTxMapping mapping handling.


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

Branch: refs/heads/ignite-2249
Commit: 26719d3c1221177616a4f291b73950f69e5ed833
Parents: cbaa6e0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Feb 8 15:49:38 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Feb 8 15:49:38 2016 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtTxPrepareFuture.java | 33 +++++-----
 ...arOptimisticSerializableTxPrepareFuture.java |  4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  2 +-
 .../GridNearPessimisticTxPrepareFuture.java     |  2 +-
 .../cache/distributed/near/GridNearTxLocal.java | 66 +++++++++++---------
 5 files changed, 56 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/26719d3c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 4c783f7..732c298 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -1202,8 +1202,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                 for (GridDistributedTxMapping nearMapping : tx.nearMap().values()) {
                     if (!tx.dhtMap().containsKey(nearMapping.node().id())) {
-                        assert nearMapping.writes() != null;
-
                         MiniFuture fut = new MiniFuture(nearMapping.node().id(), null, nearMapping);
 
                         add(fut); // Append new future.
@@ -1223,24 +1221,25 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             tx.taskNameHash(),
                             tx.activeCachesDeploymentEnabled());
 
-                        for (IgniteTxEntry entry : nearMapping.writes()) {
-                            try {
-                                if (entry.explicitVersion() == null) {
-                                    GridCacheMvccCandidate added = entry.cached().candidate(version());
+                        for (IgniteTxEntry entry : nearMapping.entries()) {
+                            if (CU.writes().apply(entry)) {
+                                try {
+                                    if (entry.explicitVersion() == null) {
+                                        GridCacheMvccCandidate added = entry.cached().candidate(version());
 
-                                assert added != null : "Null candidate for non-group-lock entry " +
-                                    "[added=" + added + ", entry=" + entry + ']';
-                                assert added.dhtLocal() : "Got non-dht-local candidate for prepare future" +
-                                    "[added=" + added + ", entry=" + entry + ']';
+                                        assert added != null : "Null candidate for non-group-lock entry " +
+                                            "[added=" + added + ", entry=" + entry + ']';
+                                        assert added.dhtLocal() : "Got non-dht-local candidate for prepare future" +
+                                            "[added=" + added + ", entry=" + entry + ']';
 
-                                    if (added != null && added.ownerVersion() != null)
-                                        req.owned(entry.txKey(), added.ownerVersion());
-                                }
+                                        if (added != null && added.ownerVersion() != null)
+                                            req.owned(entry.txKey(), added.ownerVersion());
+                                    }
 
-                                break;
-                            }
-                            catch (GridCacheEntryRemovedException ignore) {
-                                assert false : "Got removed exception on entry with dht local candidate: " + entry;
+                                    break;
+                                } catch (GridCacheEntryRemovedException ignore) {
+                                    assert false : "Got removed exception on entry with dht local candidate: " + entry;
+                                }
                             }
                         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/26719d3c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index 52ebfc8..d5483cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -443,7 +443,7 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
             m.clientFirst(),
             tx.activeCachesDeploymentEnabled());
 
-        for (IgniteTxEntry txEntry : m.writes()) {
+        for (IgniteTxEntry txEntry : m.entries()) {
             if (txEntry.op() == TRANSFORM)
                 req.addDhtVersion(txEntry.txKey(), null);
         }
@@ -451,7 +451,7 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
         // Must lock near entries separately.
         if (m.near()) {
             try {
-                tx.optimisticLockEntries(F.concat(false, m.writes(), m.reads()));
+                tx.optimisticLockEntries(m.entries());
 
                 tx.userPrepare();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/26719d3c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index b968e57..8476dc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -436,7 +436,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
                 m.clientFirst(),
                 tx.activeCachesDeploymentEnabled());
 
-            for (IgniteTxEntry txEntry : m.writes()) {
+            for (IgniteTxEntry txEntry : m.entries()) {
                 if (txEntry.op() == TRANSFORM)
                     req.addDhtVersion(txEntry.txKey(), null);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/26719d3c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 615a92b..9adf580 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -230,7 +230,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
                 false,
                 tx.activeCachesDeploymentEnabled());
 
-            for (IgniteTxEntry txEntry : m.writes()) {
+            for (IgniteTxEntry txEntry : m.entries()) {
                 if (txEntry.op() == TRANSFORM)
                     req.addDhtVersion(txEntry.txKey(), null);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/26719d3c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 0853b77..a70fb3a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -661,55 +661,61 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         Collection<GridCacheVersion> committedVers,
         Collection<GridCacheVersion> rolledbackVers)
     {
-        readyNearLocks(mapping.writes(), mapping.dhtVersion(), pendingVers, committedVers, rolledbackVers);
-        readyNearLocks(mapping.reads(), mapping.dhtVersion(), pendingVers, committedVers, rolledbackVers);
+        // Process writes, then reads.
+        for (IgniteTxEntry txEntry : mapping.entries()) {
+            if (CU.writes().apply(txEntry))
+                readyNearLock(txEntry, mapping.dhtVersion(), pendingVers, committedVers, rolledbackVers);
+        }
+
+        for (IgniteTxEntry txEntry : mapping.entries()) {
+            if (CU.reads().apply(txEntry))
+                readyNearLock(txEntry, mapping.dhtVersion(), pendingVers, committedVers, rolledbackVers);
+        }
     }
 
     /**
-     * @param entries Entries.
+     * @param txEntry TX entry.
      * @param dhtVer DHT version.
      * @param pendingVers Pending versions.
      * @param committedVers Committed versions.
      * @param rolledbackVers Rolled back versions.
      */
-    void readyNearLocks(Collection<IgniteTxEntry> entries,
+    void readyNearLock(IgniteTxEntry txEntry,
         GridCacheVersion dhtVer,
         Collection<GridCacheVersion> pendingVers,
         Collection<GridCacheVersion> committedVers,
         Collection<GridCacheVersion> rolledbackVers)
     {
-        for (IgniteTxEntry txEntry : entries) {
-            while (true) {
-                GridCacheContext cacheCtx = txEntry.cached().context();
+        while (true) {
+            GridCacheContext cacheCtx = txEntry.cached().context();
 
-                assert cacheCtx.isNear();
+            assert cacheCtx.isNear();
 
-                GridDistributedCacheEntry entry = (GridDistributedCacheEntry)txEntry.cached();
+            GridDistributedCacheEntry entry = (GridDistributedCacheEntry)txEntry.cached();
 
-                try {
-                    // Handle explicit locks.
-                    GridCacheVersion explicit = txEntry.explicitVersion();
-
-                    if (explicit == null) {
-                        entry.readyNearLock(xidVer,
-                            dhtVer,
-                            committedVers,
-                            rolledbackVers,
-                            pendingVers);
-                    }
-
-                    break;
+            try {
+                // Handle explicit locks.
+                GridCacheVersion explicit = txEntry.explicitVersion();
+
+                if (explicit == null) {
+                    entry.readyNearLock(xidVer,
+                        dhtVer,
+                        committedVers,
+                        rolledbackVers,
+                        pendingVers);
                 }
-                catch (GridCacheEntryRemovedException ignored) {
-                    assert entry.obsoleteVersion() != null;
 
-                    if (log.isDebugEnabled())
-                        log.debug("Replacing obsolete entry in remote transaction [entry=" + entry +
-                            ", tx=" + this + ']');
+                break;
+            }
+            catch (GridCacheEntryRemovedException ignored) {
+                assert entry.obsoleteVersion() != null;
 
-                    // Replace the entry.
-                    txEntry.cached(txEntry.context().cache().entryEx(txEntry.key()));
-                }
+                if (log.isDebugEnabled())
+                    log.debug("Replacing obsolete entry in remote transaction [entry=" + entry +
+                        ", tx=" + this + ']');
+
+                // Replace the entry.
+                txEntry.cached(txEntry.context().cache().entryEx(txEntry.key()));
             }
         }
     }


[07/43] ignite git commit: IGNITE-2324: .NET: Minor fixes as per static code analysis. This closes #436.

Posted by vk...@apache.org.
IGNITE-2324: .NET: Minor fixes as per static code analysis.  This closes #436.


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

Branch: refs/heads/ignite-2249
Commit: 42109899a26989aa0d2affff27d327973ea489b2
Parents: cc5067b
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Fri Feb 5 17:02:31 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Feb 5 17:02:31 2016 +0300

----------------------------------------------------------------------
 .../Cache/CachePartialUpdateException.cs        |   1 -
 .../Cache/Store/CacheStoreAdapter.cs            |   1 -
 .../Compute/ComputeTaskAdapter.cs               |   1 -
 .../Compute/ComputeTaskSplitAdapter.cs          |   1 -
 .../Apache.Ignite.Core/Events/EventBase.cs      |   1 -
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  14 +-
 .../Impl/Common/FutureType.cs                   |  18 +-
 .../Closure/ComputeAbstractClosureTask.cs       |   1 -
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |   8 +-
 .../Impl/Memory/PlatformMemoryStream.cs         | 320 +++++++++++++++----
 10 files changed, 291 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
index b80913f..907af14 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Cache
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using System.Runtime.Serialization;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Exception thrown from non-transactional cache in case when update succeeded only partially.

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
index 3f3b558..a38678d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheStoreAdapter.cs
@@ -21,7 +21,6 @@ namespace Apache.Ignite.Core.Cache.Store
     using System.Collections;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Cache storage convenience adapter. It provides default implementation for 

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
index f2d2e14..79f32a6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskAdapter.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Compute
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Convenience adapter for <see cref="IComputeTask{TArg,TJobRes,TTaskRes}"/> interface

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
index bc152b5..f6a2f07 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Compute
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl;
-    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Compute;
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
index 962d577..ed60332 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/EventBase.cs
@@ -25,7 +25,6 @@ namespace Apache.Ignite.Core.Events
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Base event implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index 7e6f418..746577a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -89,7 +89,10 @@ namespace Apache.Ignite.Core.Impl.Common
             return Task;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Set result from stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
         public void OnResult(IBinaryStream stream)
         {
@@ -103,7 +106,10 @@ namespace Apache.Ignite.Core.Impl.Common
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Set error result.
+        /// </summary>
+        /// <param name="err">Exception.</param>
         public void OnError(Exception err)
         {
             if (err is IgniteFutureCancelledException)
@@ -112,7 +118,9 @@ namespace Apache.Ignite.Core.Impl.Common
                 _taskCompletionSource.TrySetException(err);
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Set null result.
+        /// </summary>
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
         public void OnNullResult()
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
index c9f1555..c1649c9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
@@ -25,31 +25,31 @@ namespace Apache.Ignite.Core.Impl.Common
     [SuppressMessage("Microsoft.Design", "CA1008:EnumsShouldHaveZeroValue", Justification = "Interoperability")]
     public enum FutureType
     {
-        /** Future type: byte. */
+        /// <summary> Future type: byte. </summary>
         Byte = 1,
 
-        /** Future type: boolean. */
+        /// <summary> Future type: boolean. </summary>
         Bool = 2,
 
-        /** Future type: short. */
+        /// <summary> Future type: short. </summary>
         Short = 3,
 
-        /** Future type: char. */
+        /// <summary> Future type: char. </summary>
         Char = 4,
 
-        /** Future type: int. */
+        /// <summary> Future type: int. </summary>
         Int = 5,
 
-        /** Future type: float. */
+        /// <summary> Future type: float. </summary>
         Float = 6,
 
-        /** Future type: long. */
+        /// <summary> Future type: long. </summary>
         Long = 7,
 
-        /** Future type: double. */
+        /// <summary> Future type: double. </summary>
         Double = 8,
 
-        /** Future type: object. */
+        /// <summary> Future type: object. </summary>
         Object = 9
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
index 220dbf8..c967c7b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeAbstractClosureTask.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Compute;
-    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Base class for all tasks working with closures.

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
index 0168963..fb56891 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
@@ -53,14 +53,18 @@ namespace Apache.Ignite.Core.Impl.Handle
             get { return _target; }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Release the resource.
+        /// </summary>
         public void Release()
         {
             if (Interlocked.CompareExchange(ref _released, 1, 0) == 0)
                 _releaseAction(_target);
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Resource released flag.
+        /// </summary>
         public bool Released
         {
             get { return Thread.VolatileRead(ref _released) == 1; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/42109899/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
index ba0da19..5ad033e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
@@ -83,7 +83,10 @@ namespace Apache.Ignite.Core.Impl.Memory
 
         #region WRITE
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write byte.
+        /// </summary>
+        /// <param name="val">Byte value.</param>
         public void WriteByte(byte val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len1);
@@ -91,7 +94,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *(_data + curPos) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write byte array.
+        /// </summary>
+        /// <param name="val">Byte array.</param>
         public void WriteByteArray(byte[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -102,13 +108,19 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write bool.
+        /// </summary>
+        /// <param name="val">Bool value.</param>
         public void WriteBool(bool val)
         {
             WriteByte(val ? (byte)1 : (byte)0);
         }
-        
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Write bool array.
+        /// </summary>
+        /// <param name="val">Bool array.</param>
         public void WriteBoolArray(bool[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -119,7 +131,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write short.
+        /// </summary>
+        /// <param name="val">Short value.</param>
         public virtual void WriteShort(short val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len2);
@@ -127,7 +142,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((short*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write short array.
+        /// </summary>
+        /// <param name="val">Short array.</param>
         public virtual void WriteShortArray(short[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -138,7 +156,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write char.
+        /// </summary>
+        /// <param name="val">Char value.</param>
         public virtual void WriteChar(char val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len2);
@@ -146,7 +167,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((char*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write char array.
+        /// </summary>
+        /// <param name="val">Char array.</param>
         public virtual void WriteCharArray(char[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -157,7 +181,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write int.
+        /// </summary>
+        /// <param name="val">Int value.</param>
         public virtual void WriteInt(int val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len4);
@@ -165,7 +192,11 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((int*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write int to specific position.
+        /// </summary>
+        /// <param name="writePos">Position.</param>
+        /// <param name="val">Value.</param>
         [SuppressMessage("Microsoft.Usage", "CA2233:OperationsShouldNotOverflow", MessageId = "writePos+4")]
         public virtual void WriteInt(int writePos, int val)
         {
@@ -174,7 +205,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((int*)(_data + writePos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write int array.
+        /// </summary>
+        /// <param name="val">Int array.</param>
         public virtual void WriteIntArray(int[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -185,7 +219,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write long.
+        /// </summary>
+        /// <param name="val">Long value.</param>
         public virtual void WriteLong(long val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len8);
@@ -193,7 +230,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((long*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write long array.
+        /// </summary>
+        /// <param name="val">Long array.</param>
         public virtual void WriteLongArray(long[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -204,7 +244,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write float.
+        /// </summary>
+        /// <param name="val">Float value.</param>
         public virtual void WriteFloat(float val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len4);
@@ -212,7 +255,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((float*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write float array.
+        /// </summary>
+        /// <param name="val">Float array.</param>
         public virtual void WriteFloatArray(float[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -223,7 +269,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write double.
+        /// </summary>
+        /// <param name="val">Double value.</param>
         public virtual void WriteDouble(double val)
         {
             int curPos = EnsureWriteCapacityAndShift(Len8);
@@ -231,7 +280,10 @@ namespace Apache.Ignite.Core.Impl.Memory
             *((double*)(_data + curPos)) = val;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write double array.
+        /// </summary>
+        /// <param name="val">Double array.</param>
         public virtual void WriteDoubleArray(double[] val)
         {
             IgniteArgumentCheck.NotNull(val, "val");
@@ -242,7 +294,16 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write string.
+        /// </summary>
+        /// <param name="chars">Characters.</param>
+        /// <param name="charCnt">Char count.</param>
+        /// <param name="byteCnt">Byte count.</param>
+        /// <param name="encoding">Encoding.</param>
+        /// <returns>
+        /// Amounts of bytes written.
+        /// </returns>
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public int WriteString(char* chars, int charCnt, int byteCnt, Encoding encoding)
         {
@@ -254,7 +315,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return encoding.GetBytes(chars, charCnt, _data + curPos, byteCnt);
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write arbitrary data.
+        /// </summary>
+        /// <param name="src">Source array.</param>
+        /// <param name="off">Offset</param>
+        /// <param name="cnt">Count.</param>
         public void Write(byte[] src, int off, int cnt)
         {
             IgniteArgumentCheck.NotNull(src, "src");
@@ -265,25 +331,40 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Write arbitrary data.
+        /// </summary>
+        /// <param name="src">Source.</param>
+        /// <param name="cnt">Count.</param>
         public void Write(byte* src, int cnt)
         {
             CopyFromAndShift(src, cnt);
         }
-        
+
         #endregion WRITE
-        
+
         #region READ
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read byte.
+        /// </summary>
+        /// <returns>
+        /// Byte value.
+        /// </returns>
         public byte ReadByte()
         {
             int curPos = EnsureReadCapacityAndShift(Len1);
 
             return *(_data + curPos);
         }
-
-        /** <inheritdoc /> */
+        
+        /// <summary>
+        /// Read byte array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Byte array.
+        /// </returns>
         public byte[] ReadByteArray(int cnt)
         {
             int curPos = EnsureReadCapacityAndShift(cnt);
@@ -297,14 +378,25 @@ namespace Apache.Ignite.Core.Impl.Memory
 
             return res;
         }
-        
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Read bool.
+        /// </summary>
+        /// <returns>
+        /// Bool value.
+        /// </returns>
         public bool ReadBool()
         {
             return ReadByte() == 1;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read bool array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Bool array.
+        /// </returns>
         public bool[] ReadBoolArray(int cnt)
         {
             bool[] res = new bool[cnt];
@@ -317,7 +409,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read short.
+        /// </summary>
+        /// <returns>
+        /// Short value.
+        /// </returns>
         public virtual short ReadShort()
         {
             int curPos = EnsureReadCapacityAndShift(Len2);
@@ -325,7 +422,13 @@ namespace Apache.Ignite.Core.Impl.Memory
             return *((short*)(_data + curPos));
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read short array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Short array.
+        /// </returns>
         public virtual short[] ReadShortArray(int cnt)
         {
             short[] res = new short[cnt];
@@ -338,7 +441,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read char.
+        /// </summary>
+        /// <returns>
+        /// Char value.
+        /// </returns>
         public virtual char ReadChar()
         {
             int curPos = EnsureReadCapacityAndShift(Len2);
@@ -346,7 +454,13 @@ namespace Apache.Ignite.Core.Impl.Memory
             return *((char*)(_data + curPos));
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read char array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Char array.
+        /// </returns>
         public virtual char[] ReadCharArray(int cnt)
         {
             char[] res = new char[cnt];
@@ -359,15 +473,26 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read int.
+        /// </summary>
+        /// <returns>
+        /// Int value.
+        /// </returns>
         public virtual int ReadInt()
         {
             int curPos = EnsureReadCapacityAndShift(Len4);
 
             return *((int*)(_data + curPos));
         }
-        
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Read int array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Int array.
+        /// </returns>
         public virtual int[] ReadIntArray(int cnt)
         {
             int[] res = new int[cnt];
@@ -380,15 +505,26 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read long.
+        /// </summary>
+        /// <returns>
+        /// Long value.
+        /// </returns>
         public virtual long ReadLong()
         {
             int curPos = EnsureReadCapacityAndShift(Len8);
 
             return *((long*)(_data + curPos));
         }
-        
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Read long array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Long array.
+        /// </returns>
         public virtual long[] ReadLongArray(int cnt)
         {
             long[] res = new long[cnt];
@@ -401,7 +537,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read float.
+        /// </summary>
+        /// <returns>
+        /// Float value.
+        /// </returns>
         public virtual float ReadFloat()
         {
             int curPos = EnsureReadCapacityAndShift(Len4);
@@ -409,7 +550,13 @@ namespace Apache.Ignite.Core.Impl.Memory
             return *((float*)(_data + curPos));
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read float array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Float array.
+        /// </returns>
         public virtual float[] ReadFloatArray(int cnt)
         {
             float[] res = new float[cnt];
@@ -422,7 +569,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read double.
+        /// </summary>
+        /// <returns>
+        /// Double value.
+        /// </returns>
         public virtual double ReadDouble()
         {
             int curPos = EnsureReadCapacityAndShift(Len8);
@@ -430,7 +582,13 @@ namespace Apache.Ignite.Core.Impl.Memory
             return *((double*)(_data + curPos));
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read double array.
+        /// </summary>
+        /// <param name="cnt">Count.</param>
+        /// <returns>
+        /// Double array.
+        /// </returns>
         public virtual double[] ReadDoubleArray(int cnt)
         {
             double[] res = new double[cnt];
@@ -443,7 +601,12 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read arbitrary data.
+        /// </summary>
+        /// <param name="dest">Destination array.</param>
+        /// <param name="off">Offset.</param>
+        /// <param name="cnt">Count.</param>
         public void Read(byte[] dest, int off, int cnt)
         {
             IgniteArgumentCheck.NotNull(dest, "dest");
@@ -454,7 +617,11 @@ namespace Apache.Ignite.Core.Impl.Memory
             }
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Read arbitrary data.
+        /// </summary>
+        /// <param name="dest">Destination.</param>
+        /// <param name="cnt">Count.</param>
         public void Read(byte* dest, int cnt)
         {
             CopyToAndShift(dest, cnt);
@@ -516,7 +683,19 @@ namespace Apache.Ignite.Core.Impl.Memory
             _pos = 0;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Seek to the given positoin.
+        /// </summary>
+        /// <param name="offset">Offset.</param>
+        /// <param name="origin">Seek origin.</param>
+        /// <returns>
+        /// Position.
+        /// </returns>
+        /// <exception cref="System.ArgumentException">
+        /// Unsupported seek origin:  + origin
+        /// or
+        /// Seek before origin:  + newPos
+        /// </exception>
         public int Seek(int offset, SeekOrigin origin)
         {
             int newPos;
@@ -651,26 +830,41 @@ namespace Apache.Ignite.Core.Impl.Memory
             return newCap;
         }
 
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Position.
+        /// </summary>
         public int Position
         {
             get { return _pos; }
         }
 
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Gets remaining bytes in the stream.
+        /// </summary>
+        /// <value>
+        /// Remaining bytes.
+        /// </value>
         public int Remaining
         {
             get { return _len - _pos; }
         }
 
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Performs application-defined tasks associated with freeing, releasing, or resetting unmanaged resources.
+        /// </summary>
         public void Dispose()
         {
             Dispose(true);
             GC.SuppressFinalize(this);
         }
 
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Finalizes an instance of the <see cref="PlatformMemoryStream"/> class.
+        /// </summary>
         ~PlatformMemoryStream()
         {
             Dispose(false);
@@ -698,14 +892,24 @@ namespace Apache.Ignite.Core.Impl.Memory
         #endregion
 
         #region ARRAYS
-
-        /** <inheritdoc /> */
+        
+        /// <summary>
+        /// Gets underlying array, avoiding copying if possible.
+        /// </summary>
+        /// <returns>
+        /// Underlying array.
+        /// </returns>
         public byte[] GetArray()
         {
             return GetArrayCopy();
         }
-
-        /** <inheritdoc /> */
+        
+        /// <summary>
+        /// Gets underlying data in a new array.
+        /// </summary>
+        /// <returns>
+        /// New array with data.
+        /// </returns>
         public byte[] GetArrayCopy()
         {
             byte[] res = new byte[_mem.Length];
@@ -718,7 +922,13 @@ namespace Apache.Ignite.Core.Impl.Memory
             return res;
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Check whether array passed as argument is the same as the stream hosts.
+        /// </summary>
+        /// <param name="arr">Array.</param>
+        /// <returns>
+        ///   <c>True</c> if they are same.
+        /// </returns>
         public bool IsSameArray(byte[] arr)
         {
             return false;


[17/43] ignite git commit: .NET: Fixed remaining analysis warnings

Posted by vk...@apache.org.
.NET: Fixed remaining analysis warnings


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

Branch: refs/heads/ignite-2249
Commit: cbaa6e02010ee1aa8a1b4c8ee903c77ee560d8b1
Parents: e18dfda
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Mon Feb 8 14:14:30 2016 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Mon Feb 8 14:14:30 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheQueriesCodeConfigurationTest.cs           | 4 ++--
 .../Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs      | 4 ++--
 .../Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs   | 4 +++-
 .../Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs         | 6 ++++--
 4 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbaa6e02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
index a969127..684dd62 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
@@ -189,7 +189,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             /// <value>
             /// The name.
             /// </value>
-            [QueryTextField]
+            [QueryTextField(Name = "Name")]
             public string Name { get; set; }
 
             /// <summary>
@@ -258,7 +258,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             [QuerySqlField]
             public double SqlField { get; set; }
 
-            [QuerySqlField(IsIndexed = true, Name = "IndexedField1")]
+            [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true)]
             public int IndexedField { get; set; }
 
             [QueryTextField]

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbaa6e02/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index 4151540..4ce0211 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -358,8 +358,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
             if (type.IsPrimitive)
                 yield break;
 
-            var bindingFlags = BindingFlags.Public | BindingFlags.NonPublic | BindingFlags.Instance |
-                               BindingFlags.DeclaredOnly;
+            const BindingFlags bindingFlags = BindingFlags.Public | BindingFlags.NonPublic | BindingFlags.Instance |
+                                              BindingFlags.DeclaredOnly;
 
             while (type != typeof (object) && type != null)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbaa6e02/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs
index 25adf56..5df5ea1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs
@@ -124,7 +124,9 @@ namespace Apache.Ignite.Core.Discovery.Tcp.Multicast
                 writer.WriteInt(TimeToLive.Value);
         }
 
-        /** <inheritdoc /> */
+        /// <summary>
+        /// Gets the type code to be used in Java to determine ip finder type.
+        /// </summary>
         protected override byte TypeCode
         {
             get { return TypeCodeMulticastIpFinder; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbaa6e02/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs
index 331ca48..4672ed9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs
@@ -74,8 +74,10 @@ namespace Apache.Ignite.Core.Discovery.Tcp.Static
             else
                 writer.WriteInt(0);
         }
-        
-        /** <inheritdoc /> */
+
+        /// <summary>
+        /// Gets the type code to be used in Java to determine ip finder type.
+        /// </summary>
         protected override byte TypeCode
         {
             get { return TypeCodeVmIpFinder; }


[28/43] ignite git commit: ignite-2587 Fixed continuous query notifications in offheap mode and BinaryObjectOffheapImpl usage.

Posted by vk...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapValuesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapValuesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapValuesTest.java
new file mode 100644
index 0000000..2994af6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapValuesTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryTxOffheapValuesTest extends GridCacheContinuousQueryTxSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_VALUES;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 949290e..4fcc1ed 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -941,7 +941,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cacheName Cache name.
      * @return Near cache for key.
      */
-    protected IgniteCache<Integer, Integer> primaryCache(Integer key, String cacheName) {
+    protected <K, V> IgniteCache<K, V> primaryCache(Object key, String cacheName) {
         return primaryNode(key, cacheName).cache(cacheName);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 90125b1..5af37a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -86,10 +86,14 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPrimaryWrite
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPrimaryWriteOrderWithStoreInvokeTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicStopBusySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicLocalTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerEagerTtlDisabledTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxLocalTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerTxTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorCallTest;
@@ -156,9 +160,13 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheEntryListenerAtomicReplicatedTest.class);
         suite.addTestSuite(IgniteCacheEntryListenerAtomicLocalTest.class);
         suite.addTestSuite(IgniteCacheEntryListenerTxTest.class);
+        suite.addTestSuite(IgniteCacheEntryListenerTxOffheapTieredTest.class);
+        suite.addTestSuite(IgniteCacheEntryListenerTxOffheapValuesTest.class);
         suite.addTestSuite(IgniteCacheEntryListenerTxReplicatedTest.class);
         suite.addTestSuite(IgniteCacheEntryListenerTxLocalTest.class);
         suite.addTestSuite(IgniteCacheEntryListenerEagerTtlDisabledTest.class);
+        suite.addTestSuite(IgniteCacheEntryListenerAtomicOffheapTieredTest.class);
+        suite.addTestSuite(IgniteCacheEntryListenerAtomicOffheapValuesTest.class);
 
         suite.addTestSuite(IgniteClientAffinityAssignmentSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 359cdf3..3cd4579 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -70,11 +70,16 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQ
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxOffheapTieredTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
@@ -89,6 +94,8 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
@@ -181,9 +188,13 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxOffheapTieredTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxOffheapValuesTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapTieredTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapValuesTest.class);
         suite.addTestSuite(GridCacheContinuousQueryReplicatedTxOneNodeTest.class);
         suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicOneNodeTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionTxOneNodeTest.class);
@@ -195,6 +206,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
+        suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
 
         // Reduce fields queries.
         suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);


[40/43] ignite git commit: Fix R# analysis warnings

Posted by vk...@apache.org.
Fix R# analysis warnings


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

Branch: refs/heads/ignite-2249
Commit: 877be93ee35afddcc126a147cfd3cd1dda4a46ce
Parents: 0491a5f
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Feb 11 16:31:28 2016 +0300
Committer: Pavel Tupitsyn <pt...@gridgain.com>
Committed: Thu Feb 11 16:31:28 2016 +0300

----------------------------------------------------------------------
 .../Impl/Common/IgniteConfigurationXmlSerializer.cs           | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/877be93e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
index af25bfa..c27012a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
@@ -23,7 +23,6 @@ namespace Apache.Ignite.Core.Impl.Common
     using System.ComponentModel;
     using System.Configuration;
     using System.Diagnostics;
-    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using System.Reflection;
     using System.Xml;
@@ -99,14 +98,13 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Writes the property of a basic type (primitives, strings, types).
         /// </summary>
-        [SuppressMessage("ReSharper", "AssignNullToNotNullAttribute")]
         private static void WriteBasicProperty(object obj, XmlWriter writer, Type valueType, PropertyInfo property)
         {
             var converter = GetConverter(property, valueType);
 
             var stringValue = converter.ConvertToInvariantString(obj);
 
-            writer.WriteString(stringValue);
+            writer.WriteString(stringValue ?? "");
         }
 
         /// <summary>
@@ -125,7 +123,6 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Writes the complex property (nested object).
         /// </summary>
-        [SuppressMessage("ReSharper", "AssignNullToNotNullAttribute")]
         private static void WriteComplexProperty(object obj, XmlWriter writer, Type valueType)
         {
             var props = GetNonDefaultProperties(obj).ToList();
@@ -139,7 +136,7 @@ namespace Apache.Ignite.Core.Impl.Common
             {
                 var converter = GetConverter(prop, prop.PropertyType);
                 var stringValue = converter.ConvertToInvariantString(prop.GetValue(obj, null));
-                writer.WriteAttributeString(PropertyNameToXmlName(prop.Name), stringValue);
+                writer.WriteAttributeString(PropertyNameToXmlName(prop.Name), stringValue ?? "");
             }
 
             // Write elements


[29/43] ignite git commit: ignite-2587 Fixed continuous query notifications in offheap mode and BinaryObjectOffheapImpl usage.

Posted by vk...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index 27edb0c..e6bfd87 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -21,6 +21,7 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -56,6 +57,7 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
@@ -70,6 +72,7 @@ import static javax.cache.event.EventType.CREATED;
 import static javax.cache.event.EventType.EXPIRED;
 import static javax.cache.event.EventType.REMOVED;
 import static javax.cache.event.EventType.UPDATED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -79,7 +82,7 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED;
  */
 public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAbstractTest {
     /** */
-    private static volatile List<CacheEntryEvent<? extends Integer, ? extends Integer>> evts;
+    private static volatile List<CacheEntryEvent<?, ?>> evts;
 
     /** */
     private static volatile CountDownLatch evtsLatch;
@@ -91,7 +94,10 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     private Integer lastKey = 0;
 
     /** */
-    private CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg;
+    private CacheEntryListenerConfiguration<Object, Object> lsnrCfg;
+
+    /** */
+    private boolean useObjects;
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
@@ -103,9 +109,18 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         cfg.setEagerTtl(eagerTtl());
 
+        cfg.setMemoryMode(memoryMode());
+
         return cfg;
     }
 
+    /**
+     * @return Cache memory mode.
+     */
+    protected CacheMemoryMode memoryMode() {
+        return ONHEAP_TIERED;
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
@@ -129,9 +144,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @throws Exception If failed.
      */
     public void testExceptionIgnored() throws Exception {
-        CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
-            new Factory<CacheEntryListener<Integer, Integer>>() {
-                @Override public CacheEntryListener<Integer, Integer> create() {
+        CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+            new Factory<CacheEntryListener<Object, Object>>() {
+                @Override public CacheEntryListener<Object, Object> create() {
                     return new ExceptionListener();
                 }
             },
@@ -140,7 +155,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             false
         );
 
-        IgniteCache<Integer, Integer> cache = jcache();
+        IgniteCache<Object, Object> cache = jcache();
 
         cache.registerCacheEntryListener(lsnrCfg);
 
@@ -158,13 +173,13 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         }
 
         lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
-            new Factory<CacheEntryListener<Integer, Integer>>() {
-                @Override public CacheEntryListener<Integer, Integer> create() {
+            new Factory<CacheEntryListener<Object, Object>>() {
+                @Override public CacheEntryListener<Object, Object> create() {
                     return new CreateUpdateRemoveExpireListener();
                 }
             },
-            new Factory<CacheEntryEventSerializableFilter<? super Integer, ? super Integer>>() {
-                @Override public CacheEntryEventSerializableFilter<? super Integer, ? super Integer> create() {
+            new Factory<CacheEntryEventSerializableFilter<Object, Object>>() {
+                @Override public CacheEntryEventSerializableFilter<Object, Object> create() {
                     return new ExceptionFilter();
                 }
             },
@@ -192,9 +207,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @throws Exception If failed.
      */
     public void testNoOldValue() throws Exception {
-        CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
-            new Factory<CacheEntryListener<Integer, Integer>>() {
-                @Override public CacheEntryListener<Integer, Integer> create() {
+        CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+            new Factory<CacheEntryListener<Object, Object>>() {
+                @Override public CacheEntryListener<Object, Object> create() {
                     return new CreateUpdateRemoveExpireListener();
                 }
             },
@@ -203,7 +218,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             true
         );
 
-        IgniteCache<Integer, Integer> cache = jcache();
+        IgniteCache<Object, Object> cache = jcache();
 
         try {
             for (Integer key : keys()) {
@@ -222,21 +237,30 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      * @throws Exception If failed.
      */
+    public void testSynchronousEventsObjectKeyValue() throws Exception {
+        useObjects = true;
+
+        testSynchronousEvents();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testSynchronousEvents() throws Exception {
-        final CacheEntryCreatedListener<Integer, Integer> lsnr = new CreateUpdateRemoveExpireListener() {
-            @Override public void onRemoved(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+        final CacheEntryCreatedListener<Object, Object> lsnr = new CreateUpdateRemoveExpireListener() {
+            @Override public void onRemoved(Iterable<CacheEntryEvent<?, ?>> evts) {
                 super.onRemoved(evts);
 
                 awaitLatch();
             }
 
-            @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+            @Override public void onCreated(Iterable<CacheEntryEvent<?, ?>> evts) {
                 super.onCreated(evts);
 
                 awaitLatch();
             }
 
-            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
                 super.onUpdated(evts);
 
                 awaitLatch();
@@ -252,9 +276,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             }
         };
 
-        CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
-            new Factory<CacheEntryListener<Integer, Integer>>() {
-                @Override public CacheEntryListener<Integer, Integer> create() {
+        CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+            new Factory<CacheEntryListener<Object, Object>>() {
+                @Override public CacheEntryListener<Object, Object> create() {
                     return lsnr;
                 }
             },
@@ -263,7 +287,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             true
         );
 
-        IgniteCache<Integer, Integer> cache = jcache();
+        IgniteCache<Object, Object> cache = jcache();
 
         cache.registerCacheEntryListener(lsnrCfg);
 
@@ -299,7 +323,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
                 if (!eagerTtl()) {
                     U.sleep(1100);
 
-                    assertNull(primaryCache(key, cache.getName()).get(key));
+                    assertNull(primaryCache(key, cache.getName()).get(key(key)));
 
                     evtsLatch.await(5000, MILLISECONDS);
 
@@ -378,13 +402,13 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         final CyclicBarrier barrier = new CyclicBarrier(THREADS);
 
-        final IgniteCache<Integer, Integer> cache = jcache(0);
+        final IgniteCache<Object, Object> cache = jcache(0);
 
         GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
-                CacheEntryListenerConfiguration<Integer, Integer> cfg = new MutableCacheEntryListenerConfiguration<>(
-                    new Factory<CacheEntryListener<Integer, Integer>>() {
-                        @Override public CacheEntryListener<Integer, Integer> create() {
+                CacheEntryListenerConfiguration<Object, Object> cfg = new MutableCacheEntryListenerConfiguration<>(
+                    new Factory<CacheEntryListener<Object, Object>>() {
+                        @Override public CacheEntryListener<Object, Object> create() {
                             return new CreateUpdateRemoveExpireListener();
                         }
                     },
@@ -441,9 +465,13 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @param expEvts Expected events number.
      * @throws Exception If failed.
      */
-    private void syncEvent(Integer key, Integer val, IgniteCache<Integer, Integer> cache, int expEvts)
+    private void syncEvent(
+        Integer key,
+        Integer val,
+        IgniteCache<Object, Object> cache,
+        int expEvts)
         throws Exception {
-        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<? extends Integer, ? extends Integer>>());
+        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<?, ?>>());
 
         evtsLatch = new CountDownLatch(expEvts);
 
@@ -466,9 +494,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         });
 
         if (val != null)
-            cache.put(key, val);
+            cache.put(key(key), value(val));
         else
-            cache.remove(key);
+            cache.remove(key(key));
 
         done.set(true);
 
@@ -480,15 +508,45 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     }
 
     /**
+     * @param key Integer key.
+     * @return Key instance.
+     */
+    private Object key(Integer key) {
+        assert key != null;
+
+        return useObjects ? new ListenerTestKey(key) : key;
+    }
+
+    /**
+     * @param val Integer value.
+     * @return Value instance.
+     */
+    private Object value(Integer val) {
+        if (val == null)
+            return null;
+
+        return useObjects ? new ListenerTestValue(val) : val;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEventsObjectKeyValue() throws Exception {
+        useObjects = true;
+
+        testEvents();
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testEvents() throws Exception {
-        IgniteCache<Integer, Integer> cache = jcache();
+        IgniteCache<Object, Object> cache = jcache();
 
-        Map<Integer, Integer> vals = new HashMap<>();
+        Map<Object, Object> vals = new HashMap<>();
 
         for (int i = 0; i < 100; i++)
-            vals.put(i + 2_000_000, i);
+            vals.put(key(i + 2_000_000), value(i));
 
         cache.putAll(vals); // Put some data in cache to make sure events are not generated for existing entries.
 
@@ -518,7 +576,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             checkEvents(cache, new CreateUpdateRemoveExpireListenerFactory(), key, true, true, true, true);
         }
 
-        CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+        CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
             new CreateUpdateRemoveExpireListenerFactory(),
             new TestFilterFactory(),
             true,
@@ -551,7 +609,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private void checkListenerOnStart(Map<Integer, Integer> vals) throws Exception {
+    private void checkListenerOnStart(Map<Object, Object> vals) throws Exception {
         lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
             new CreateUpdateRemoveExpireListenerFactory(),
             null,
@@ -564,7 +622,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         try {
             awaitPartitionMapExchange();
 
-            IgniteCache<Integer, Integer> cache = grid.cache(null);
+            IgniteCache<Object, Object> cache = grid.cache(null);
 
             Integer key = Integer.MAX_VALUE;
 
@@ -588,7 +646,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         try {
             awaitPartitionMapExchange();
 
-            IgniteCache<Integer, Integer> cache = grid.cache(null);
+            IgniteCache<Object, Object> cache = grid.cache(null);
 
             log.info("Check filter for listener in configuration.");
 
@@ -613,14 +671,14 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @throws Exception If failed.
      */
     private void checkEvents(
-        final IgniteCache<Integer, Integer> cache,
-        final Factory<CacheEntryListener<Integer, Integer>> lsnrFactory,
+        final IgniteCache<Object, Object> cache,
+        final Factory<CacheEntryListener<Object, Object>> lsnrFactory,
         Integer key,
         boolean create,
         boolean update,
         boolean rmv,
         boolean expire) throws Exception {
-        CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+        CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
             lsnrFactory,
             null,
             true,
@@ -642,8 +700,8 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @param vals Values in cache.
      * @throws Exception If failed.
      */
-    private void checkFilter(final IgniteCache<Integer, Integer> cache, Map<Integer, Integer> vals) throws Exception {
-        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<? extends Integer, ? extends Integer>>());
+    private void checkFilter(final IgniteCache<Object, Object> cache, Map<Object, Object> vals) throws Exception {
+        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<?, ?>>());
 
         final int expEvts = (vals.size() / 2) * 4; // Remove, create, update and expire for half of modified entries.
 
@@ -653,16 +711,18 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         cache.putAll(vals);
 
-        final Map<Integer, Integer> newVals = new HashMap<>();
+        final Map<Object, Object> newVals = new HashMap<>();
 
-        for (Integer key : vals.keySet())
-            newVals.put(key, -1);
+        for (Object key : vals.keySet())
+            newVals.put(key, value(-1));
 
         cache.withExpiryPolicy(new ModifiedExpiryPolicy(new Duration(MILLISECONDS, 500))).putAll(newVals);
 
+        U.sleep(1000);
+
         GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
-                for (Integer key : newVals.keySet()) {
+                for (Object key : newVals.keySet()) {
                     if (primaryCache(key, cache.getName()).get(key) != null)
                         return false;
                 }
@@ -675,13 +735,20 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         assertEquals(expEvts, evts.size());
 
-        Set<Integer> rmvd = new HashSet<>();
-        Set<Integer> created = new HashSet<>();
-        Set<Integer> updated = new HashSet<>();
-        Set<Integer> expired = new HashSet<>();
+        Set<Object> rmvd = new HashSet<>();
+        Set<Object> created = new HashSet<>();
+        Set<Object> updated = new HashSet<>();
+        Set<Object> expired = new HashSet<>();
+
+        for (CacheEntryEvent<?, ?> evt : evts) {
+            Integer key;
+
+            if (useObjects)
+                key = ((ListenerTestKey)evt.getKey()).key;
+            else
+                key = (Integer)evt.getKey();
 
-        for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts) {
-            assertTrue(evt.getKey() % 2 == 0);
+            assertTrue(key % 2 == 0);
 
             assertTrue(vals.keySet().contains(evt.getKey()));
 
@@ -707,7 +774,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
                     break;
 
                 case UPDATED:
-                    assertEquals(-1, (int)evt.getValue());
+                    assertEquals(value(-1), evt.getValue());
 
                     assertEquals(vals.get(evt.getKey()), evt.getOldValue());
 
@@ -722,7 +789,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
                 case EXPIRED:
                     assertNull(evt.getValue());
 
-                    assertEquals(-1, (int)evt.getOldValue());
+                    assertEquals(value(-1), evt.getOldValue());
 
                     assertTrue(rmvd.contains(evt.getKey()));
 
@@ -757,8 +824,8 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @throws Exception If failed.
      */
     private void checkEvents(
-        final IgniteCache<Integer, Integer> cache,
-        final CacheEntryListenerConfiguration<Integer, Integer> lsnrCfg,
+        final IgniteCache<Object, Object> cache,
+        final CacheEntryListenerConfiguration<Object, Object> lsnrCfg,
         Integer key,
         boolean create,
         boolean update,
@@ -789,64 +856,64 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         if (expire)
             expEvts += 2;
 
-        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<? extends Integer, ? extends Integer>>());
+        evts = Collections.synchronizedList(new ArrayList<CacheEntryEvent<?, ?>>());
 
         evtsLatch = new CountDownLatch(expEvts);
 
-        cache.put(key, 0);
+        cache.put(key(key), value(0));
 
         for (int i = 0; i < UPDATES; i++) {
             if (i % 2 == 0)
-                cache.put(key, i + 1);
+                cache.put(key(key), value(i + 1));
             else
-                cache.invoke(key, new EntrySetValueProcessor(i + 1));
+                cache.invoke(key(key), new EntrySetValueProcessor(value(i + 1)));
         }
 
         // Invoke processor does not update value, should not trigger event.
-        assertEquals(String.valueOf(UPDATES), cache.invoke(key, new EntryToStringProcessor()));
+        assertEquals(String.valueOf(UPDATES), cache.invoke(key(key), new EntryToStringProcessor()));
 
-        assertFalse(cache.putIfAbsent(key, -1));
+        assertFalse(cache.putIfAbsent(key(key), value(-1)));
 
-        assertFalse(cache.remove(key, -1));
+        assertFalse(cache.remove(key(key), value(-1)));
 
-        assertTrue(cache.remove(key));
+        assertTrue(cache.remove(key(key)));
 
-        IgniteCache<Integer, Integer> expirePlcCache =
+        IgniteCache<Object, Object> expirePlcCache =
             cache.withExpiryPolicy(new CreatedExpiryPolicy(new Duration(MILLISECONDS, 100)));
 
-        expirePlcCache.put(key, 10);
+        expirePlcCache.put(key(key), value(10));
 
         U.sleep(700);
 
         if (!eagerTtl())
-            assertNull(primaryCache(key, cache.getName()).get(key)); // Provoke expire event if eager ttl is disabled.
+            assertNull(primaryCache(key, cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled.
 
-        IgniteCache<Integer, Integer> cache1 = cache;
+        IgniteCache<Object, Object> cache1 = cache;
 
         if (gridCount() > 1)
             cache1 = jcache(1); // Do updates from another node.
 
-        cache1.put(key, 1);
+        cache1.put(key(key), value(1));
 
-        cache1.put(key, 2);
+        cache1.put(key(key), value(2));
 
-        assertTrue(cache1.remove(key));
+        assertTrue(cache1.remove(key(key)));
 
-        IgniteCache<Integer, Integer> expirePlcCache1 =
+        IgniteCache<Object, Object> expirePlcCache1 =
             cache1.withExpiryPolicy(new CreatedExpiryPolicy(new Duration(MILLISECONDS, 100)));
 
-        expirePlcCache1.put(key, 20);
+        expirePlcCache1.put(key(key), value(20));
 
         U.sleep(200);
 
         if (!eagerTtl())
-            assertNull(primaryCache(key, cache.getName()).get(key)); // Provoke expire event if eager ttl is disabled.
+            assertNull(primaryCache(key, cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled.
 
         evtsLatch.await(5000, MILLISECONDS);
 
         assertEquals(expEvts, evts.size());
 
-        Iterator<CacheEntryEvent<? extends Integer, ? extends Integer>> iter = evts.iterator();
+        Iterator<CacheEntryEvent<?, ?>> iter = evts.iterator();
 
         if (create)
             checkEvent(iter, key, CREATED, 0, null);
@@ -886,11 +953,11 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         cache.deregisterCacheEntryListener(lsnrCfg);
 
-        cache.put(key, 1);
+        cache.put(key(key), value(1));
 
-        cache.put(key, 2);
+        cache.put(key(key), value(2));
 
-        assertTrue(cache.remove(key));
+        assertTrue(cache.remove(key(key)));
 
         U.sleep(500); // Sleep some time to ensure listener was really removed.
 
@@ -908,26 +975,26 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      * @param expVal Expected value.
      * @param expOld Expected old value.
      */
-    private void checkEvent(Iterator<CacheEntryEvent<? extends Integer, ? extends Integer>> iter,
+    private void checkEvent(Iterator<CacheEntryEvent<?, ?>> iter,
         Integer expKey,
         EventType expType,
         @Nullable Integer expVal,
         @Nullable Integer expOld) {
         assertTrue(iter.hasNext());
 
-        CacheEntryEvent<? extends Integer, ? extends Integer> evt = iter.next();
+        CacheEntryEvent<?, ?> evt = iter.next();
 
         iter.remove();
 
         assertTrue(evt.getSource() instanceof IgniteCacheProxy);
 
-        assertEquals(expKey, evt.getKey());
+        assertEquals(key(expKey), evt.getKey());
 
         assertEquals(expType, evt.getEventType());
 
-        assertEquals(expVal, evt.getValue());
+        assertEquals(value(expVal), evt.getValue());
 
-        assertEquals(expOld, evt.getOldValue());
+        assertEquals(value(expOld), evt.getOldValue());
 
         if (expOld == null)
             assertFalse(evt.isOldValueAvailable());
@@ -977,7 +1044,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      * @param evt Event.
      */
-    private static void onEvent(CacheEntryEvent<? extends Integer, ? extends Integer> evt) {
+    private static void onEvent(CacheEntryEvent<?, ?> evt) {
         // System.out.println("Received event [evt=" + evt + ", thread=" + Thread.currentThread().getName() + ']');
 
         assertNotNull(evt);
@@ -993,9 +1060,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class CreateUpdateRemoveExpireListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class CreateUpdateRemoveExpireListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new CreateUpdateRemoveExpireListener();
         }
     }
@@ -1003,9 +1070,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class NoOpCreateUpdateListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class NoOpCreateUpdateListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new NoOpCreateUpdateListener();
         }
     }
@@ -1013,9 +1080,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class CreateUpdateListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class CreateUpdateListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new CreateUpdateListener();
         }
     }
@@ -1023,9 +1090,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class CreateListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class CreateListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new CreateListener();
         }
     }
@@ -1033,9 +1100,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class RemoveListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class RemoveListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new RemoveListener();
         }
     }
@@ -1043,9 +1110,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class UpdateListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class UpdateListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new UpdateListener();
         }
     }
@@ -1053,9 +1120,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class ExpireListenerFactory implements Factory<CacheEntryListener<Integer, Integer>> {
+    private static class ExpireListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryListener<Integer, Integer> create() {
+        @Override public CacheEntryListener<Object, Object> create() {
             return new ExpireListener();
         }
     }
@@ -1063,9 +1130,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class TestFilterFactory implements Factory<CacheEntryEventSerializableFilter<Integer, Integer>> {
+    private static class TestFilterFactory implements Factory<CacheEntryEventSerializableFilter<Object, Object>> {
         /** {@inheritDoc} */
-        @Override public CacheEntryEventSerializableFilter<Integer, Integer> create() {
+        @Override public CacheEntryEventSerializableFilter<Object, Object> create() {
             return new TestFilter();
         }
     }
@@ -1073,10 +1140,10 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class CreateListener implements CacheEntryCreatedListener<Integer, Integer> {
+    private static class CreateListener implements CacheEntryCreatedListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onCreated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1084,10 +1151,10 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class UpdateListener implements CacheEntryUpdatedListener<Integer, Integer> {
+    private static class UpdateListener implements CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1095,10 +1162,10 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class RemoveListener implements CacheEntryRemovedListener<Integer, Integer> {
+    private static class RemoveListener implements CacheEntryRemovedListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onRemoved(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onRemoved(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1106,10 +1173,10 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class ExpireListener implements CacheEntryExpiredListener<Integer, Integer> {
+    private static class ExpireListener implements CacheEntryExpiredListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onExpired(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onExpired(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1117,32 +1184,39 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class TestFilter implements CacheEntryEventSerializableFilter<Integer, Integer> {
+    private static class TestFilter implements CacheEntryEventSerializableFilter<Object, Object> {
         /** {@inheritDoc} */
-        @Override public boolean evaluate(CacheEntryEvent<? extends Integer, ? extends Integer> evt) {
+        @Override public boolean evaluate(CacheEntryEvent<?, ?> evt) {
             assert evt != null;
             assert evt.getSource() != null : evt;
             assert evt.getEventType() != null : evt;
             assert evt.getKey() != null : evt;
 
-            return evt.getKey() % 2 == 0;
+            Integer key;
+
+            if (evt.getKey() instanceof ListenerTestKey)
+                key = ((ListenerTestKey)evt.getKey()).key;
+            else
+                key = (Integer)evt.getKey();
+
+            return key % 2 == 0;
         }
     }
 
     /**
      *
      */
-    private static class CreateUpdateListener implements CacheEntryCreatedListener<Integer, Integer>,
-        CacheEntryUpdatedListener<Integer, Integer> {
+    private static class CreateUpdateListener implements CacheEntryCreatedListener<Object, Object>,
+        CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onCreated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
 
         /** {@inheritDoc} */
-        @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1150,11 +1224,11 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class NoOpCreateUpdateListener implements CacheEntryCreatedListener<Integer, Integer>,
-        CacheEntryUpdatedListener<Integer, Integer> {
+    private static class NoOpCreateUpdateListener implements CacheEntryCreatedListener<Object, Object>,
+        CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts) {
+        @Override public void onCreated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts) {
                 assertNotNull(evt);
                 assertNotNull(evt.getSource());
                 assertNotNull(evt.getEventType());
@@ -1163,8 +1237,8 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         }
 
         /** {@inheritDoc} */
-        @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts) {
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts) {
                 assertNotNull(evt);
                 assertNotNull(evt.getSource());
                 assertNotNull(evt.getEventType());
@@ -1177,16 +1251,16 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      *
      */
     private static class CreateUpdateRemoveExpireListener extends CreateUpdateListener
-        implements CacheEntryRemovedListener<Integer, Integer>, CacheEntryExpiredListener<Integer, Integer> {
+        implements CacheEntryRemovedListener<Object, Object>, CacheEntryExpiredListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onRemoved(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onRemoved(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
 
         /** {@inheritDoc} */
-        @Override public void onExpired(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
-            for (CacheEntryEvent<? extends Integer, ? extends Integer> evt : evts)
+        @Override public void onExpired(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
                 onEvent(evt);
         }
     }
@@ -1194,9 +1268,9 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    private static class ExceptionFilter implements CacheEntryEventSerializableFilter<Integer, Integer> {
+    private static class ExceptionFilter implements CacheEntryEventSerializableFilter<Object, Object> {
         /** {@inheritDoc} */
-        @Override public boolean evaluate(CacheEntryEvent<? extends Integer, ? extends Integer> evt) {
+        @Override public boolean evaluate(CacheEntryEvent<?, ?> evt) {
             throw new RuntimeException("Test filter error.");
         }
     }
@@ -1205,24 +1279,24 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
      *
      */
     private static class ExceptionListener extends CreateUpdateListener
-        implements CacheEntryRemovedListener<Integer, Integer>, CacheEntryExpiredListener<Integer, Integer> {
+        implements CacheEntryRemovedListener<Object, Object>, CacheEntryExpiredListener<Object, Object> {
         /** {@inheritDoc} */
-        @Override public void onCreated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+        @Override public void onCreated(Iterable<CacheEntryEvent<?, ?>> evts) {
             error();
         }
 
         /** {@inheritDoc} */
-        @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
             error();
         }
 
         /** {@inheritDoc} */
-        @Override public void onExpired(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+        @Override public void onExpired(Iterable<CacheEntryEvent<?, ?>> evts) {
             error();
         }
 
         /** {@inheritDoc} */
-        @Override public void onRemoved(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+        @Override public void onRemoved(Iterable<CacheEntryEvent<?, ?>> evts) {
             error();
         }
 
@@ -1237,10 +1311,12 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    protected static class EntryToStringProcessor implements EntryProcessor<Integer, Integer, String> {
+    protected static class EntryToStringProcessor implements EntryProcessor<Object, Object, String> {
         /** {@inheritDoc} */
-        @Override public String process(MutableEntry<Integer, Integer> e, Object... arguments)
-            throws EntryProcessorException {
+        @Override public String process(MutableEntry<Object, Object> e, Object... args) {
+            if (e.getValue() instanceof ListenerTestValue)
+                return String.valueOf(((ListenerTestValue)e.getValue()).val1);
+
             return String.valueOf(e.getValue());
         }
 
@@ -1253,19 +1329,19 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
     /**
      *
      */
-    protected static class EntrySetValueProcessor implements EntryProcessor<Integer, Integer, String> {
+    protected static class EntrySetValueProcessor implements EntryProcessor<Object, Object, String> {
         /** */
-        private Integer val;
+        private Object val;
 
         /**
          * @param val Value to set.
          */
-        public EntrySetValueProcessor(Integer val) {
+        public EntrySetValueProcessor(Object val) {
             this.val = val;
         }
 
         /** {@inheritDoc} */
-        @Override public String process(MutableEntry<Integer, Integer> e, Object... arguments)
+        @Override public String process(MutableEntry<Object, Object> e, Object... args)
             throws EntryProcessorException {
             e.setValue(val);
 
@@ -1307,4 +1383,88 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
             // No-op.
         }
     }
+
+    /**
+     *
+     */
+    static class ListenerTestKey implements Serializable {
+        /** */
+        private final Integer key;
+
+        /**
+         * @param key Key.
+         */
+        public ListenerTestKey(Integer key) {
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            ListenerTestKey that = (ListenerTestKey)o;
+
+            return key.equals(that.key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ListenerTestKey.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ListenerTestValue implements Serializable {
+        /** */
+        private final Integer val1;
+
+        /** */
+        private final String val2;
+
+        /**
+         * @param val Value.
+         */
+        public ListenerTestValue(Integer val) {
+            this.val1 = val;
+            this.val2 = String.valueOf(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            ListenerTestValue that = (ListenerTestValue) o;
+
+            return val1.equals(that.val1) && val2.equals(that.val2);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = val1.hashCode();
+
+            res = 31 * res + val2.hashCode();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ListenerTestValue.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapTieredTest.java
new file mode 100644
index 0000000..69efb84
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapTieredTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class IgniteCacheEntryListenerAtomicOffheapTieredTest extends IgniteCacheEntryListenerAtomicTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapValuesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapValuesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapValuesTest.java
new file mode 100644
index 0000000..23b1bc0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAtomicOffheapValuesTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheEntryListenerAtomicOffheapValuesTest extends IgniteCacheEntryListenerAtomicTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_VALUES;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapTieredTest.java
new file mode 100644
index 0000000..d552195
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapTieredTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class IgniteCacheEntryListenerTxOffheapTieredTest extends IgniteCacheEntryListenerTxTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapValuesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapValuesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapValuesTest.java
new file mode 100644
index 0000000..32555c8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxOffheapValuesTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+
+/**
+ *
+ */
+public class IgniteCacheEntryListenerTxOffheapValuesTest extends IgniteCacheEntryListenerTxTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_VALUES;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxTest.java
index a9e43d4..41725e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerTxTest.java
@@ -48,6 +48,7 @@ public class IgniteCacheEntryListenerTxTest extends IgniteCacheEntryListenerAbst
         return null;
     }
 
+    /** {@inheritDoc} */
     @Override public void testEvents(){
         fail("https://issues.apache.org/jira/browse/IGNITE-1600");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 1c65f9b..a42f056 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -55,6 +55,7 @@ import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.ContinuousQuery;
@@ -97,6 +98,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
@@ -142,6 +144,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
         ccfg.setBackups(backups);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
         ccfg.setNearConfiguration(nearCacheConfiguration());
+        ccfg.setMemoryMode(memoryMode());
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -151,6 +154,13 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
     }
 
     /**
+     * @return Cache memory mode.
+     */
+    protected CacheMemoryMode memoryMode() {
+        return ONHEAP_TIERED;
+    }
+
+    /**
      * @return Near cache configuration.
      */
     protected NearCacheConfiguration nearCacheConfiguration() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.java
new file mode 100644
index 0000000..cc8590d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest
+    extends CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxOffheapTieredTest.java
new file mode 100644
index 0000000..cae06c3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxOffheapTieredTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class CacheContinuousQueryFailoverTxOffheapTieredTest extends CacheContinuousQueryFailoverTxSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
new file mode 100644
index 0000000..d9b2091
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
@@ -0,0 +1,684 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import java.io.Serializable;
+import java.util.Random;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryUpdatedListener;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 5;
+
+    /** */
+    private static final int KEYS = 10;
+
+    /** */
+    private static final int VALS = 10;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(NODES - 1);
+
+        client = true;
+
+        startGrid(NODES - 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomic() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReplicated() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            ATOMIC,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNoBackups() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            ATOMIC,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxReplicated() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_VALUES,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL,
+            OFFHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxNoBackups() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL,
+            ONHEAP_TIERED,
+            false);
+
+        testContinuousQuery(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void testContinuousQuery(CacheConfiguration<Object, Object> ccfg) throws Exception {
+        ignite(0).createCache(ccfg);
+
+        try {
+            IgniteCache<Object, Object> cache = ignite(NODES - 1).cache(ccfg.getName());
+
+            long seed = System.currentTimeMillis();
+
+            Random rnd = new Random(seed);
+
+            log.info("Random seed: " + seed);
+
+            ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+            final BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue =
+                new ArrayBlockingQueue<>(10_000);
+
+            qry.setLocalListener(new CacheEntryUpdatedListener<Object, Object>() {
+                @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+                    for (CacheEntryEvent<?, ?> evt : evts) {
+                        // System.out.println("Event: " + evt);
+
+                        evtsQueue.add(evt);
+                    }
+                }
+            });
+
+            QueryCursor<?> cur = cache.query(qry);
+
+            ConcurrentMap<Object, Object> expData = new ConcurrentHashMap<>();
+
+            try {
+                for (int i = 0; i < 1000; i++) {
+                    if (i % 100 == 0)
+                        log.info("Iteration: " + i);
+
+                    randomUpdate(rnd, evtsQueue, expData, cache);
+                }
+            }
+            finally {
+                cur.close();
+            }
+        }
+        finally {
+            ignite(0).destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @param rnd Random generator.
+     * @param evtsQueue Events queue.
+     * @param expData Expected cache data.
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void randomUpdate(
+        Random rnd,
+        BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue,
+        ConcurrentMap<Object, Object> expData,
+        IgniteCache<Object, Object> cache)
+        throws Exception {
+        Object key = new QueryTestKey(rnd.nextInt(KEYS));
+        Object newVal = value(rnd);
+        Object oldVal = expData.get(key);
+
+        int op = rnd.nextInt(11);
+
+        // log.info("Random operation [key=" + key + ", op=" + op + ']');
+
+        switch (op) {
+            case 0: {
+                cache.put(key, newVal);
+
+                waitEvent(evtsQueue, key, newVal, oldVal);
+
+                expData.put(key, newVal);
+
+                break;
+            }
+
+            case 1: {
+                cache.getAndPut(key, newVal);
+
+                waitEvent(evtsQueue, key, newVal, oldVal);
+
+                expData.put(key, newVal);
+
+                break;
+            }
+
+            case 2: {
+                cache.remove(key);
+
+                waitEvent(evtsQueue, key, null, oldVal);
+
+                expData.remove(key);
+
+                break;
+            }
+
+            case 3: {
+                cache.getAndRemove(key);
+
+                waitEvent(evtsQueue, key, null, oldVal);
+
+                expData.remove(key);
+
+                break;
+            }
+
+            case 4: {
+                cache.invoke(key, new EntrySetValueProcessor(newVal, rnd.nextBoolean()));
+
+                waitEvent(evtsQueue, key, newVal, oldVal);
+
+                expData.put(key, newVal);
+
+                break;
+            }
+
+            case 5: {
+                cache.invoke(key, new EntrySetValueProcessor(null, rnd.nextBoolean()));
+
+                waitEvent(evtsQueue, key, null, oldVal);
+
+                expData.remove(key);
+
+                break;
+            }
+
+            case 6: {
+                cache.putIfAbsent(key, newVal);
+
+                if (oldVal == null) {
+                    waitEvent(evtsQueue, key, newVal, null);
+
+                    expData.put(key, newVal);
+                }
+                else
+                    checkNoEvent(evtsQueue);
+
+                break;
+            }
+
+            case 7: {
+                cache.getAndPutIfAbsent(key, newVal);
+
+                if (oldVal == null) {
+                    waitEvent(evtsQueue, key, newVal, null);
+
+                    expData.put(key, newVal);
+                }
+                else
+                    checkNoEvent(evtsQueue);
+
+                break;
+            }
+
+            case 8: {
+                cache.replace(key, newVal);
+
+                if (oldVal != null) {
+                    waitEvent(evtsQueue, key, newVal, oldVal);
+
+                    expData.put(key, newVal);
+                }
+                else
+                    checkNoEvent(evtsQueue);
+
+                break;
+            }
+
+            case 9: {
+                cache.getAndReplace(key, newVal);
+
+                if (oldVal != null) {
+                    waitEvent(evtsQueue, key, newVal, oldVal);
+
+                    expData.put(key, newVal);
+                }
+                else
+                    checkNoEvent(evtsQueue);
+
+                break;
+            }
+
+            case 10: {
+                if (oldVal != null) {
+                    Object replaceVal = value(rnd);
+
+                    boolean success = replaceVal.equals(oldVal);
+
+                    if (success) {
+                        cache.replace(key, replaceVal, newVal);
+
+                        waitEvent(evtsQueue, key, newVal, oldVal);
+
+                        expData.put(key, newVal);
+                    }
+                    else {
+                        cache.replace(key, replaceVal, newVal);
+
+                        checkNoEvent(evtsQueue);
+                    }
+                }
+                else {
+                    cache.replace(key, value(rnd), newVal);
+
+                    checkNoEvent(evtsQueue);
+                }
+
+                break;
+            }
+
+            default:
+                fail();
+        }
+    }
+
+    /**
+     * @param rnd Random generator.
+     * @return Cache value.
+     */
+    private static Object value(Random rnd) {
+        return new QueryTestValue(rnd.nextInt(VALS));
+    }
+
+    /**
+     * @param evtsQueue Event queue.
+     * @param key Key.
+     * @param val Value.
+     * @param oldVal Old value.
+     * @throws Exception If failed.
+     */
+    private void waitEvent(BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue,
+        Object key, Object val, Object oldVal) throws Exception {
+        if (val == null && oldVal == null) {
+            checkNoEvent(evtsQueue);
+
+            return;
+        }
+
+        CacheEntryEvent<?, ?> evt = evtsQueue.poll(5, SECONDS);
+
+        assertNotNull("Failed to wait for event [key=" + key +
+            ", val=" + val +
+            ", oldVal=" + oldVal + ']', evt);
+        assertEquals(key, evt.getKey());
+        assertEquals(val, evt.getValue());
+        assertEquals(oldVal, evt.getOldValue());
+    }
+
+    /**
+     * @param evtsQueue Event queue.
+     * @throws Exception If failed.
+     */
+    private void checkNoEvent(BlockingQueue<CacheEntryEvent<?, ?>> evtsQueue) throws Exception {
+        CacheEntryEvent<?, ?> evt = evtsQueue.poll(50, MILLISECONDS);
+
+        assertNull(evt);
+    }
+
+    /**
+     *
+     * @param cacheMode Cache mode.
+     * @param backups Number of backups.
+     * @param atomicityMode Cache atomicity mode.
+     * @param memoryMode Cache memory mode.
+     * @param store If {@code true} configures dummy cache store.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration(
+        CacheMode cacheMode,
+        int backups,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode,
+        boolean store) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setMemoryMode(memoryMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        if (store) {
+            ccfg.setCacheStoreFactory(new TestStoreFactory());
+            ccfg.setReadThrough(true);
+            ccfg.setWriteThrough(true);
+        }
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    private static class TestStoreFactory implements Factory<CacheStore<Object, Object>> {
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public CacheStore<Object, Object> create() {
+            return new CacheStoreAdapter() {
+                @Override public Object load(Object key) throws CacheLoaderException {
+                    return null;
+                }
+
+                @Override public void write(Cache.Entry entry) throws CacheWriterException {
+                    // No-op.
+                }
+
+                @Override public void delete(Object key) throws CacheWriterException {
+                    // No-op.
+                }
+            };
+        }
+    }
+
+    /**
+     *
+     */
+    static class QueryTestKey implements Serializable {
+        /** */
+        private final Integer key;
+
+        /**
+         * @param key Key.
+         */
+        public QueryTestKey(Integer key) {
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            QueryTestKey that = (QueryTestKey)o;
+
+            return key.equals(that.key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(QueryTestKey.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class QueryTestValue implements Serializable {
+        /** */
+        private final Integer val1;
+
+        /** */
+        private final String val2;
+
+        /**
+         * @param val Value.
+         */
+        public QueryTestValue(Integer val) {
+            this.val1 = val;
+            this.val2 = String.valueOf(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            QueryTestValue that = (QueryTestValue) o;
+
+            return val1.equals(that.val1) && val2.equals(that.val2);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = val1.hashCode();
+
+            res = 31 * res + val2.hashCode();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(QueryTestValue.class, this);
+        }
+    }
+    /**
+     *
+     */
+    protected static class EntrySetValueProcessor implements EntryProcessor<Object, Object, Object> {
+        /** */
+        private Object val;
+
+        /** */
+        private boolean retOld;
+
+        /**
+         * @param val Value to set.
+         * @param retOld Return old value flag.
+         */
+        public EntrySetValueProcessor(Object val, boolean retOld) {
+            this.val = val;
+            this.retOld = retOld;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Object, Object> e, Object... args) {
+            Object old = retOld ? e.getValue() : null;
+
+            if (val != null)
+                e.setValue(val);
+            else
+                e.remove();
+
+            return old;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(EntrySetValueProcessor.class, this);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 5abb98d..dbe282e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
@@ -73,9 +74,9 @@ import org.jsr166.ConcurrentHashMap8;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -117,6 +118,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             cacheCfg.setReadThrough(true);
             cacheCfg.setWriteThrough(true);
             cacheCfg.setLoadPreviousValue(true);
+            cacheCfg.setMemoryMode(memoryMode());
 
             cfg.setCacheConfiguration(cacheCfg);
         }
@@ -135,6 +137,13 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
     }
 
     /**
+     * @return Cache memory mode.
+     */
+    protected CacheMemoryMode memoryMode() {
+        return ONHEAP_TIERED;
+    }
+
+    /**
      * @return Peer class loading enabled flag.
      */
     protected boolean peerClassLoadingEnabled() {
@@ -393,8 +402,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
         });
 
-        try (QueryCursor<Cache.Entry<Integer, Integer>> query2 = cache1.query(qry2);
-            QueryCursor<Cache.Entry<Integer, Integer>> query1 = cache.query(qry1)) {
+        try (QueryCursor<Cache.Entry<Integer, Integer>> qryCur2 = cache1.query(qry2);
+             QueryCursor<Cache.Entry<Integer, Integer>> qryCur1 = cache.query(qry1)) {
             for (int i = 0; i < gridCount(); i++) {
                 IgniteCache<Object, Object> cache0 = grid(i).cache(null);
 
@@ -448,7 +457,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                     }
                 });
 
-                QueryCursor<Cache.Entry<Integer, Integer>> query = cache.query(qry);
+                QueryCursor<Cache.Entry<Integer, Integer>> qryCur = cache.query(qry);
 
                 for (int key = 0; key < keyCnt; key++)
                     cache.put(key, key);
@@ -461,7 +470,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                     }, 2000L);
                 }
                 finally {
-                    query.close();
+                    qryCur.close();
                 }
             }
             else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapTieredTest.java
new file mode 100644
index 0000000..d6948e2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapTieredTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryAtomicOffheapTieredTest extends GridCacheContinuousQueryAtomicSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapValuesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapValuesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapValuesTest.java
new file mode 100644
index 0000000..4002435
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAtomicOffheapValuesTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryAtomicOffheapValuesTest extends GridCacheContinuousQueryAtomicSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_VALUES;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c05fc02/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapTieredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapTieredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapTieredTest.java
new file mode 100644
index 0000000..bcba7b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryTxOffheapTieredTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.cache.CacheMemoryMode;
+
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+
+/**
+ *
+ */
+public class GridCacheContinuousQueryTxOffheapTieredTest extends GridCacheContinuousQueryTxSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMemoryMode memoryMode() {
+        return OFFHEAP_TIERED;
+    }
+}


[02/43] ignite git commit: Renamed args array.

Posted by vk...@apache.org.
Renamed args array.


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

Branch: refs/heads/ignite-2249
Commit: a30f84aaece33e81ab420a6466a516ebf220366a
Parents: 532b373
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Feb 5 11:21:56 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Feb 5 11:21:56 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAdapter.java       | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a30f84aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2c3a197..9f54ddb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -5545,7 +5545,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         private final IgniteBiPredicate<K, V> p;
 
         /** */
-        private final Object[] args;
+        private final Object[] loadArgs;
 
         /** */
         private final ExpiryPolicy plc;
@@ -5554,15 +5554,15 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
          * @param cacheName Cache name.
          * @param topVer Affinity topology version.
          * @param p Predicate.
-         * @param args Arguments.
+         * @param loadArgs Arguments.
          * @param plc Policy.
          */
-        private LoadCacheJob(String cacheName, AffinityTopologyVersion topVer, IgniteBiPredicate<K, V> p, Object[] args,
+        private LoadCacheJob(String cacheName, AffinityTopologyVersion topVer, IgniteBiPredicate<K, V> p, Object[] loadArgs,
             ExpiryPolicy plc) {
             super(cacheName, topVer);
 
             this.p = p;
-            this.args = args;
+            this.loadArgs = loadArgs;
             this.plc = plc;
         }
 
@@ -5574,7 +5574,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (plc != null)
                     cache = cache.withExpiryPolicy(plc);
 
-                cache.localLoadCache(p, args);
+                cache.localLoadCache(p, loadArgs);
 
                 return null;
             }


[35/43] ignite git commit: IGNITE-2195: Implemented Hadoop FileSystem factory capable of working with kerberized file systems. This closes #464.

Posted by vk...@apache.org.
IGNITE-2195: Implemented Hadoop FileSystem factory capable of working with kerberized file systems. This closes #464.


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

Branch: refs/heads/ignite-2249
Commit: 008c8cd3f33b9c2cace43a9d1f2b4e4542fb58fe
Parents: fa3706f
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Feb 10 15:45:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 15:45:58 2016 +0300

----------------------------------------------------------------------
 .../hadoop/fs/BasicHadoopFileSystemFactory.java |  22 +-
 .../fs/CachingHadoopFileSystemFactory.java      |   2 +-
 .../fs/KerberosHadoopFileSystemFactory.java     | 214 +++++++++++++++++++
 ...KerberosHadoopFileSystemFactorySelfTest.java | 104 +++++++++
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 5 files changed, 339 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/008c8cd3/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
index c791e9a..01fe6c9 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
@@ -66,7 +66,7 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex
 
     /** {@inheritDoc} */
     @Override public FileSystem get(String usrName) throws IOException {
-        return create0(IgfsUtils.fixUserName(usrName));
+        return get0(IgfsUtils.fixUserName(usrName));
     }
 
     /**
@@ -76,7 +76,7 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex
      * @return File system.
      * @throws IOException If failed.
      */
-    protected FileSystem create0(String usrName) throws IOException {
+    protected FileSystem get0(String usrName) throws IOException {
         assert cfg != null;
 
         try {
@@ -87,12 +87,12 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex
             ClassLoader clsLdr = getClass().getClassLoader();
 
             if (ctxClsLdr == clsLdr)
-                return FileSystem.get(fullUri, cfg, usrName);
+                return create(usrName);
             else {
                 Thread.currentThread().setContextClassLoader(clsLdr);
 
                 try {
-                    return FileSystem.get(fullUri, cfg, usrName);
+                    return create(usrName);
                 }
                 finally {
                     Thread.currentThread().setContextClassLoader(ctxClsLdr);
@@ -107,6 +107,18 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex
     }
 
     /**
+     * Internal file system creation routine, invoked in correct class loader context.
+     *
+     * @param usrName User name.
+     * @return File system.
+     * @throws IOException If failed.
+     * @throws InterruptedException if the current thread is interrupted.
+     */
+    protected FileSystem create(String usrName) throws IOException, InterruptedException {
+        return FileSystem.get(fullUri, cfg, usrName);
+    }
+
+    /**
      * Gets file system URI.
      * <p>
      * This URI will be used as a first argument when calling {@link FileSystem#get(URI, Configuration, String)}.
@@ -152,7 +164,7 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex
      *
      * @param cfgPaths Paths to file system configuration files.
      */
-    public void setConfigPaths(String... cfgPaths) {
+    public void setConfigPaths(@Nullable String... cfgPaths) {
         this.cfgPaths = cfgPaths;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/008c8cd3/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
index 91f7777..e1b30c4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
@@ -47,7 +47,7 @@ public class CachingHadoopFileSystemFactory extends BasicHadoopFileSystemFactory
     private final transient HadoopLazyConcurrentMap<String, FileSystem> cache = new HadoopLazyConcurrentMap<>(
         new HadoopLazyConcurrentMap.ValueFactory<String, FileSystem>() {
             @Override public FileSystem createValue(String key) throws IOException {
-                return create0(key);
+                return get0(key);
             }
         }
     );

http://git-wip-us.apache.org/repos/asf/ignite/blob/008c8cd3/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
new file mode 100644
index 0000000..fc768d6
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.hadoop.fs;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.security.PrivilegedExceptionAction;
+
+/**
+ * Secure Hadoop file system factory that can work with underlying file system protected with Kerberos.
+ * It uses "impersonation" mechanism, to be able to work on behalf of arbitrary client user.
+ * Please see https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Superusers.html for details.
+ * The principal and the key tab name to be used for Kerberos authentication are set explicitly
+ * in the factory configuration.
+ *
+ * <p>This factory does not cache any file system instances. If {@code "fs.[prefix].impl.disable.cache"} is set
+ * to {@code true}, file system instances will be cached by Hadoop.
+ */
+public class KerberosHadoopFileSystemFactory extends BasicHadoopFileSystemFactory {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** The default interval used to re-login from the key tab, in milliseconds. */
+    public static final long DFLT_RELOGIN_INTERVAL = 10 * 60 * 1000L;
+
+    /** Keytab full file name. */
+    private String keyTab;
+
+    /** Keytab principal. */
+    private String keyTabPrincipal;
+
+    /** The re-login interval. See {@link #getReloginInterval()} for more information. */
+    private long reloginInterval = DFLT_RELOGIN_INTERVAL;
+
+    /** Time of last re-login attempt, in system milliseconds. */
+    private transient volatile long lastReloginTime;
+
+    /**
+     * Constructor.
+     */
+    public KerberosHadoopFileSystemFactory() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileSystem get(String userName) throws IOException {
+        reloginIfNeeded();
+
+        return super.get(userName);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FileSystem create(String usrName) throws IOException, InterruptedException {
+        UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser(usrName,
+            UserGroupInformation.getLoginUser());
+
+        return proxyUgi.doAs(new PrivilegedExceptionAction<FileSystem>() {
+            @Override public FileSystem run() throws Exception {
+                return FileSystem.get(fullUri, cfg);
+            }
+        });
+    }
+
+    /**
+     * Gets the key tab principal short name (e.g. "hdfs").
+     *
+     * @return The key tab principal.
+     */
+    @Nullable public String getKeyTabPrincipal() {
+        return keyTabPrincipal;
+    }
+
+    /**
+     * Set the key tab principal name. See {@link #getKeyTabPrincipal()} for more information.
+     *
+     * @param keyTabPrincipal The key tab principal name.
+     */
+    public void setKeyTabPrincipal(@Nullable String keyTabPrincipal) {
+        this.keyTabPrincipal = keyTabPrincipal;
+    }
+
+    /**
+     * Gets the key tab full file name (e.g. "/etc/security/keytabs/hdfs.headless.keytab" or "/etc/krb5.keytab").
+     *
+     * @return The key tab file name.
+     */
+    @Nullable public String getKeyTab() {
+        return keyTab;
+    }
+
+    /**
+     * Sets the key tab file name. See {@link #getKeyTab()} for more information.
+     *
+     * @param keyTab The key tab file name.
+     */
+    public void setKeyTab(@Nullable String keyTab) {
+        this.keyTab = keyTab;
+    }
+
+    /**
+     * The interval used to re-login from the key tab, in milliseconds.
+     * Important that the value should not be larger than the Kerberos ticket life time multiplied by 0.2. This is
+     * because the ticket renew window starts from {@code 0.8 * ticket life time}.
+     * Default ticket life time is 1 day (24 hours), so the default re-login interval (10 min)
+     * is obeys this rule well.
+     *
+     * <p>Zero value means that re-login should be attempted on each file system operation.
+     * Negative values are not allowed.
+     *
+     * <p>Note, however, that it does not make sense to make this value small, because Hadoop does not allow to
+     * login if less than {@link org.apache.hadoop.security.UserGroupInformation#MIN_TIME_BEFORE_RELOGIN} milliseconds
+     * have passed since the time of the previous login.
+     * See {@link org.apache.hadoop.security.UserGroupInformation#hasSufficientTimeElapsed(long)} and its usages for
+     * more detail.
+     *
+     * @return The re-login interval, in milliseconds.
+     */
+    public long getReloginInterval() {
+        return reloginInterval;
+    }
+
+    /**
+     * Sets the relogin interval in milliseconds. See {@link #getReloginInterval()} for more information.
+     *
+     * @param reloginInterval The re-login interval, in milliseconds.
+     */
+    public void setReloginInterval(long reloginInterval) {
+        this.reloginInterval = reloginInterval;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        A.ensure(!F.isEmpty(keyTab), "keyTab cannot not be empty.");
+        A.ensure(!F.isEmpty(keyTabPrincipal), "keyTabPrincipal cannot not be empty.");
+        A.ensure(reloginInterval >= 0, "reloginInterval cannot not be negative.");
+
+        super.start();
+
+        try {
+            UserGroupInformation.setConfiguration(cfg);
+            UserGroupInformation.loginUserFromKeytab(keyTabPrincipal, keyTab);
+        }
+        catch (IOException ioe) {
+            throw new IgniteException("Failed login from keytab [keyTab=" + keyTab +
+                ", keyTabPrincipal=" + keyTabPrincipal + ']', ioe);
+        }
+    }
+
+    /**
+     * Re-logins the user if needed.
+     * First, the re-login interval defined in factory is checked. The re-login attempts will be not more
+     * frequent than one attempt per {@code reloginInterval}.
+     * Second, {@link UserGroupInformation#checkTGTAndReloginFromKeytab()} method invoked that gets existing
+     * TGT and checks its validity. If the TGT is expired or is close to expiry, it performs re-login.
+     *
+     * <p>This operation expected to be called upon each operation with the file system created with the factory.
+     * As long as {@link #get(String)} operation is invoked upon each file {@link IgniteHadoopFileSystem}, there
+     * is no need to invoke it otherwise specially.
+     *
+     * @throws IOException If login fails.
+     */
+    private void reloginIfNeeded() throws IOException {
+        long now = System.currentTimeMillis();
+
+        if (now >= lastReloginTime + reloginInterval) {
+            UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
+
+            lastReloginTime = now;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        U.writeString(out, keyTab);
+        U.writeString(out, keyTabPrincipal);
+        out.writeLong(reloginInterval);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        keyTab = U.readString(in);
+        keyTabPrincipal = U.readString(in);
+        reloginInterval = in.readLong();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/008c8cd3/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
new file mode 100644
index 0000000..8fb1612
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
@@ -0,0 +1,104 @@
+package org.apache.ignite.hadoop.fs;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.util.concurrent.Callable;
+
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+/**
+ * Tests KerberosHadoopFileSystemFactory.
+ */
+public class KerberosHadoopFileSystemFactorySelfTest extends GridCommonAbstractTest {
+    /**
+     * Test parameters validation.
+     *
+     * @throws Exception If failed.
+     */
+    public void testParameters() throws Exception {
+        checkParameters(null, null, -1);
+
+        checkParameters(null, null, 100);
+        checkParameters(null, "b", -1);
+        checkParameters("a", null, -1);
+
+        checkParameters(null, "b", 100);
+        checkParameters("a", null, 100);
+        checkParameters("a", "b", -1);
+    }
+
+    /**
+     * Check parameters.
+     *
+     * @param keyTab Key tab.
+     * @param keyTabPrincipal Key tab principal.
+     * @param reloginInterval Re-login interval.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    private void checkParameters(String keyTab, String keyTabPrincipal, long reloginInterval) {
+        final KerberosHadoopFileSystemFactory fac = new KerberosHadoopFileSystemFactory();
+
+        fac.setKeyTab(keyTab);
+        fac.setKeyTabPrincipal(keyTabPrincipal);
+        fac.setReloginInterval(reloginInterval);
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fac.start();
+
+                return null;
+            }
+        }, IllegalArgumentException.class, null);
+    }
+
+    /**
+     * Checks serializatuion and deserialization of the secure factory.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSerialization() throws Exception {
+        KerberosHadoopFileSystemFactory fac = new KerberosHadoopFileSystemFactory();
+
+        checkSerialization(fac);
+
+        fac = new KerberosHadoopFileSystemFactory();
+
+        fac.setUri("igfs://igfs@localhost:10500/");
+        fac.setConfigPaths("/a/core-sute.xml", "/b/mapred-site.xml");
+        fac.setKeyTabPrincipal("foo");
+        fac.setKeyTab("/etc/krb5.keytab");
+        fac.setReloginInterval(30 * 60 * 1000L);
+
+        checkSerialization(fac);
+    }
+
+    /**
+     * Serializes the factory,
+     *
+     * @param fac The facory to check.
+     * @throws Exception If failed.
+     */
+    private void checkSerialization(KerberosHadoopFileSystemFactory fac) throws Exception {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        ObjectOutput oo = new ObjectOutputStream(baos);
+
+        oo.writeObject(fac);
+
+        ObjectInput in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray()));
+
+        KerberosHadoopFileSystemFactory fac2 = (KerberosHadoopFileSystemFactory)in.readObject();
+
+        assertEquals(fac.getUri(), fac2.getUri());
+        Assert.assertArrayEquals(fac.getConfigPaths(), fac2.getConfigPaths());
+        assertEquals(fac.getKeyTab(), fac2.getKeyTab());
+        assertEquals(fac.getKeyTabPrincipal(), fac2.getKeyTabPrincipal());
+        assertEquals(fac.getReloginInterval(), fac2.getReloginInterval());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/008c8cd3/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 9092f32..acd255c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -35,6 +35,7 @@ import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.client.hadoop.HadoopClientProtocolEmbeddedSelfTest;
 import org.apache.ignite.client.hadoop.HadoopClientProtocolSelfTest;
+import org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactorySelfTest;
 import org.apache.ignite.igfs.Hadoop1OverIgfsDualAsyncTest;
 import org.apache.ignite.igfs.Hadoop1OverIgfsDualSyncTest;
 import org.apache.ignite.igfs.HadoopFIleSystemFactorySelfTest;
@@ -101,6 +102,8 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 
+        suite.addTest(new TestSuite(ldr.loadClass(KerberosHadoopFileSystemFactorySelfTest.class.getName())));
+
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyFullMapReduceTest.class.getName())));
 


[12/43] ignite git commit: IGNITE-1759: .NET: Improved GUID handling for different platforms and endians. This closes #437.

Posted by vk...@apache.org.
IGNITE-1759: .NET: Improved GUID handling for different platforms and endians. This closes #437.


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

Branch: refs/heads/ignite-2249
Commit: f07adff7bb601971bb6c83b0459113678f387592
Parents: e88cc67
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Mon Feb 8 10:30:57 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Feb 8 10:30:57 2016 +0300

----------------------------------------------------------------------
 .../Binary/BinarySelfTest.cs                    |  32 ++++
 .../Impl/Binary/BinaryUtils.cs                  | 166 ++++++++++++++++---
 2 files changed, 172 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f07adff7/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 44db6f7..f49a28a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -26,12 +26,15 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System.Collections;
     using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
+    using System.IO;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
     using NUnit.Framework;
+    using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
+    using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
     /// 
@@ -476,6 +479,35 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(vals, newVals);
         }
 
+        /// <summary>
+        /// Checks that both methods produce identical results.
+        /// </summary>
+        [Test]
+        public void TestGuidSlowFast()
+        {
+            var stream = new BinaryHeapStream(128);
+
+            var guid = Guid.NewGuid();
+
+            BinaryUtils.WriteGuidFast(guid, stream);
+
+            stream.Seek(0, SeekOrigin.Begin);
+            Assert.AreEqual(guid, BinaryUtils.ReadGuidFast(stream));
+
+            stream.Seek(0, SeekOrigin.Begin);
+            Assert.AreEqual(guid, BinaryUtils.ReadGuidSlow(stream));
+
+
+            stream.Seek(0, SeekOrigin.Begin);
+            BinaryUtils.WriteGuidFast(guid, stream);
+
+            stream.Seek(0, SeekOrigin.Begin);
+            Assert.AreEqual(guid, BinaryUtils.ReadGuidFast(stream));
+
+            stream.Seek(0, SeekOrigin.Begin);
+            Assert.AreEqual(guid, BinaryUtils.ReadGuidSlow(stream));
+        }
+
         /**
         * <summary>Check write of enum.</summary>
         */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f07adff7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 06dec2c..9066bd1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -243,6 +243,17 @@ namespace Apache.Ignite.Core.Impl.Binary
         private static readonly CopyOnWriteConcurrentDictionary<Type, Func<BinaryReader, bool, object>>
             ArrayReaders = new CopyOnWriteConcurrentDictionary<Type, Func<BinaryReader, bool, object>>();
 
+        /** Flag indicating whether Guid struct is sequential in current runtime. */
+        private static readonly bool IsGuidSequential = GetIsGuidSequential();
+
+        /** Guid writer. */
+        public static readonly Action<Guid, IBinaryStream> WriteGuid = IsGuidSequential
+            ? (Action<Guid, IBinaryStream>)WriteGuidFast : WriteGuidSlow;
+
+        /** Guid reader. */
+        public static readonly Func<IBinaryStream, Guid?> ReadGuid = IsGuidSequential
+            ? (Func<IBinaryStream, Guid?>)ReadGuidFast : ReadGuidSlow;
+
         /// <summary>
         /// Default marshaller.
         /// </summary>
@@ -900,12 +911,33 @@ namespace Apache.Ignite.Core.Impl.Binary
             return vals;
         }
 
-        /**
-         * <summary>Write GUID.</summary>
-         * <param name="val">GUID.</param>
-         * <param name="stream">Stream.</param>
-         */
-        public static unsafe void WriteGuid(Guid val, IBinaryStream stream)
+        /// <summary>
+        /// Gets a value indicating whether <see cref="Guid"/> fields are stored sequentially in memory.
+        /// </summary>
+        /// <returns></returns>
+        private static unsafe bool GetIsGuidSequential()
+        {
+            // Check that bitwise conversion returns correct result
+            var guid = Guid.NewGuid();
+
+            var bytes = guid.ToByteArray();
+
+            var bytes0 = (byte*) &guid;
+
+            for (var i = 0; i < bytes.Length; i++)
+                if (bytes[i] != bytes0[i])
+                    return false;
+
+            return true;
+        }
+
+        /// <summary>
+        /// Writes a guid with bitwise conversion, assuming that <see cref="Guid"/> 
+        /// is laid out in memory sequentially and without gaps between fields.
+        /// </summary>
+        /// <param name="val">The value.</param>
+        /// <param name="stream">The stream.</param>
+        public static unsafe void WriteGuidFast(Guid val, IBinaryStream stream)
         {
             var jguid = new JavaGuid(val);
 
@@ -913,13 +945,47 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             stream.Write((byte*) ptr, 16);
         }
-        
-        /**
-         * <summary>Read GUID.</summary>
-         * <param name="stream">Stream.</param>
-         * <returns>GUID</returns>
-         */
-        public static unsafe Guid? ReadGuid(IBinaryStream stream)
+
+        /// <summary>
+        /// Writes a guid byte by byte.
+        /// </summary>
+        /// <param name="val">The value.</param>
+        /// <param name="stream">The stream.</param>
+        public static unsafe void WriteGuidSlow(Guid val, IBinaryStream stream)
+        {
+            var bytes = val.ToByteArray();
+            byte* jBytes = stackalloc byte[16];
+
+            jBytes[0] = bytes[6]; // c1
+            jBytes[1] = bytes[7]; // c2
+
+            jBytes[2] = bytes[4]; // b1
+            jBytes[3] = bytes[5]; // b2
+
+            jBytes[4] = bytes[0]; // a1
+            jBytes[5] = bytes[1]; // a2
+            jBytes[6] = bytes[2]; // a3
+            jBytes[7] = bytes[3]; // a4
+
+            jBytes[8] = bytes[15]; // k
+            jBytes[9] = bytes[14]; // j
+            jBytes[10] = bytes[13]; // i
+            jBytes[11] = bytes[12]; // h
+            jBytes[12] = bytes[11]; // g
+            jBytes[13] = bytes[10]; // f
+            jBytes[14] = bytes[9]; // e
+            jBytes[15] = bytes[8]; // d
+            
+            stream.Write(jBytes, 16);
+        }
+
+        /// <summary>
+        /// Reads a guid with bitwise conversion, assuming that <see cref="Guid"/> 
+        /// is laid out in memory sequentially and without gaps between fields.
+        /// </summary>
+        /// <param name="stream">The stream.</param>
+        /// <returns>Guid.</returns>
+        public static unsafe Guid? ReadGuidFast(IBinaryStream stream)
         {
             JavaGuid jguid;
 
@@ -931,7 +997,43 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             return *(Guid*) (&dotnetGuid);
         }
-        
+
+        /// <summary>
+        /// Reads a guid byte by byte.
+        /// </summary>
+        /// <param name="stream">The stream.</param>
+        /// <returns>Guid.</returns>
+        public static unsafe Guid? ReadGuidSlow(IBinaryStream stream)
+        {
+            byte* jBytes = stackalloc byte[16];
+
+            stream.Read(jBytes, 16);
+
+            var bytes = new byte[16];
+
+            bytes[0] = jBytes[4]; // a1
+            bytes[1] = jBytes[5]; // a2
+            bytes[2] = jBytes[6]; // a3
+            bytes[3] = jBytes[7]; // a4
+
+            bytes[4] = jBytes[2]; // b1
+            bytes[5] = jBytes[3]; // b2
+
+            bytes[6] = jBytes[0]; // c1
+            bytes[7] = jBytes[1]; // c2
+
+            bytes[8] = jBytes[15]; // d
+            bytes[9] = jBytes[14]; // e
+            bytes[10] = jBytes[13]; // f
+            bytes[11] = jBytes[12]; // g
+            bytes[12] = jBytes[11]; // h
+            bytes[13] = jBytes[10]; // i
+            bytes[14] = jBytes[9]; // j
+            bytes[15] = jBytes[8]; // k
+
+            return new Guid(bytes);
+        }
+
         /// <summary>
         /// Write GUID array.
         /// </summary>
@@ -1689,7 +1791,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         private struct GuidAccessor
         {
             public readonly ulong ABC;
-            public readonly ulong DEGHIJK;
+            public readonly ulong DEFGHIJK;
 
             /// <summary>
             /// Initializes a new instance of the <see cref="GuidAccessor"/> struct.
@@ -1699,21 +1801,28 @@ namespace Apache.Ignite.Core.Impl.Binary
             {
                 var l = val.CBA;
 
-                ABC = ((l >> 32) & 0x00000000FFFFFFFF) | ((l << 48) & 0xFFFF000000000000) |
-                      ((l << 16) & 0x0000FFFF00000000);
+                if (BitConverter.IsLittleEndian)
+                    ABC = ((l >> 32) & 0x00000000FFFFFFFF) | ((l << 48) & 0xFFFF000000000000) |
+                          ((l << 16) & 0x0000FFFF00000000);
+                else
+                    ABC = ((l << 32) & 0xFFFFFFFF00000000) | ((l >> 48) & 0x000000000000FFFF) |
+                          ((l >> 16) & 0x00000000FFFF0000);
 
-                DEGHIJK = ReverseByteOrder(val.KJIHGED);
+                // This is valid in any endianness (symmetrical)
+                DEFGHIJK = ReverseByteOrder(val.KJIHGFED);
             }
         }
 
         /// <summary>
         /// Struct with Java-style Guid memory layout.
         /// </summary>
-        [StructLayout(LayoutKind.Sequential, Pack = 0)]
+        [StructLayout(LayoutKind.Explicit)]
         private struct JavaGuid
         {
-            public readonly ulong CBA;
-            public readonly ulong KJIHGED;
+            [FieldOffset(0)] public readonly ulong CBA;
+            [FieldOffset(8)] public readonly ulong KJIHGFED;
+            [SuppressMessage("Microsoft.Performance", "CA1823:AvoidUnusedPrivateFields")]
+            [FieldOffset(0)] public unsafe fixed byte Bytes [16];
 
             /// <summary>
             /// Initializes a new instance of the <see cref="JavaGuid"/> struct.
@@ -1721,17 +1830,22 @@ namespace Apache.Ignite.Core.Impl.Binary
             /// <param name="val">The value.</param>
             public unsafe JavaGuid(Guid val)
             {
-                // .Net returns bytes in the following order: _a(4), _b(2), _c(2), _d, _e, _g, _h, _i, _j, _k.
+                // .Net returns bytes in the following order: _a(4), _b(2), _c(2), _d, _e, _f, _g, _h, _i, _j, _k.
                 // And _a, _b and _c are always in little endian format irrespective of system configuration.
-                // To be compliant with Java we rearrange them as follows: _c, _b_, a_, _k, _j, _i, _h, _g, _e, _d.
+                // To be compliant with Java we rearrange them as follows: _c, _b_, a_, _k, _j, _i, _h, _g, _f, _e, _d.
                 var accessor = *((GuidAccessor*)&val);
 
                 var l = accessor.ABC;
 
-                CBA = ((l << 32) & 0xFFFFFFFF00000000) | ((l >> 48) & 0x000000000000FFFF) |
-                      ((l >> 16) & 0x00000000FFFF0000);
+                if (BitConverter.IsLittleEndian)
+                    CBA = ((l << 32) & 0xFFFFFFFF00000000) | ((l >> 48) & 0x000000000000FFFF) |
+                          ((l >> 16) & 0x00000000FFFF0000);
+                else
+                    CBA = ((l >> 32) & 0x00000000FFFFFFFF) | ((l << 48) & 0xFFFF000000000000) |
+                          ((l << 16) & 0x0000FFFF00000000);
 
-                KJIHGED = ReverseByteOrder(accessor.DEGHIJK);
+                // This is valid in any endianness (symmetrical)
+                KJIHGFED = ReverseByteOrder(accessor.DEFGHIJK);
             }
         }
     }


[14/43] ignite git commit: Improved CacheStore + external class loader tests

Posted by vk...@apache.org.
Improved CacheStore + external class loader tests


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

Branch: refs/heads/ignite-2249
Commit: bf3a5ea652a9c36e5ce8dd87dba72b5fafc3ce86
Parents: bad0420
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Feb 8 13:41:38 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Feb 8 13:41:38 2016 +0300

----------------------------------------------------------------------
 .../GridCacheReplicatedPreloadSelfTest.java     | 120 +++++++++++++------
 1 file changed, 82 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf3a5ea6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index 523f641..887fea4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -77,6 +77,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     private volatile boolean isClient = false;
 
     /** */
+    private volatile boolean useExtClassLoader = false;
+
+    /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
@@ -108,7 +111,8 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setEventStorageSpi(spi);
 
-        if (getTestGridName(1).equals(gridName) || cfg.getMarshaller() instanceof BinaryMarshaller)
+        if (getTestGridName(1).equals(gridName) || useExtClassLoader ||
+            cfg.getMarshaller() instanceof BinaryMarshaller)
             cfg.setClassLoader(getExternalClassLoader());
 
         if (isClient)
@@ -187,7 +191,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
             for (int i = 0; i < 3; i++) {
                 evts = g2.events().localQuery(F.<Event>alwaysTrue(),
-                        EVT_CACHE_REBALANCE_STARTED, EVT_CACHE_REBALANCE_STOPPED);
+                    EVT_CACHE_REBALANCE_STARTED, EVT_CACHE_REBALANCE_STOPPED);
 
                 if (evts.size() != 2) {
                     info("Wrong events collection size (will retry in 1000 ms): " + evts.size());
@@ -284,7 +288,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assert v3 != null;
             assert v3.toString().equals(v1.toString());
             assert !v3.getClass().getClassLoader().equals(getClass().getClassLoader());
-            assert v3.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader")||
+            assert v3.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader") ||
                 grid(3).configuration().getMarshaller() instanceof BinaryMarshaller;
         }
         finally {
@@ -297,70 +301,110 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
      */
     public void testStore() throws Exception {
         try {
+            needStore = true;
+            useExtClassLoader = true;
+
             Ignite g1 = startGrid(1);
 
-            if (g1.configuration().getMarshaller() instanceof BinaryMarshaller) {
-                stopAllGrids();
+            Ignite g2 = startGrid(2);  // Checks deserialization at node join.
 
-                needStore = true;
+            isClient = true;
 
-                g1 = startGrid(1);
+            Ignite g3 = startGrid(3);
+
+            IgniteCache<Integer, Object> cache1 = g1.cache(null);
+            IgniteCache<Integer, Object> cache2 = g2.cache(null);
+            IgniteCache<Integer, Object> cache3 = g3.cache(null);
+
+            cache1.put(1, 1);
+
+            assertEquals(1, cache2.get(1));
+            assertEquals(1, cache3.get(1));
+        }
+        finally {
+            needStore = false;
+            isClient = false;
+            useExtClassLoader = false;
+        }
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testStoreDynamicStart() throws Exception {
+        try {
+            needStore = false;
+            useExtClassLoader = true;
 
-                ClassLoader ldr = grid(1).configuration().getClassLoader();
+            Ignite g1 = startGrid(1);
+            Ignite g2 = startGrid(2);
 
-                CacheConfiguration cfg = defaultCacheConfiguration();
+            isClient = true;
 
-                Ignite g2 = startGrid(2);  // Checks deserialization at node join.
+            Ignite g3 = startGrid(3);
 
-                isClient = true;
+            Object sf = getExternalClassLoader().loadClass(
+                "org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
 
-                Ignite g3 = startGrid(3);
+            CacheConfiguration cfg = defaultCacheConfiguration();
 
-                isClient = false;
+            cfg.setCacheStoreFactory((Factory)sf);
+            cfg.setName("customStore");
 
-                IgniteCache<Integer, Object> cache1 = g1.cache(null);
-                IgniteCache<Integer, Object> cache2 = g2.cache(null);
-                IgniteCache<Integer, Object> cache3 = g3.cache(null);
+            IgniteCache<Integer, Object> cache1 = g1.createCache(cfg);
 
-                cache1.put(1, 1);
+            IgniteCache<Integer, Object> cache2 = g2.getOrCreateCache(cfg); // Checks deserialization at cache creation.
+            IgniteCache<Integer, Object> cache3 = g3.getOrCreateCache(cfg); // Checks deserialization at cache creation.
 
-                assertEquals(1, cache2.get(1));
-                assertEquals(1, cache3.get(1));
+            cache1.put(1, 1);
 
-                needStore = false;
+            assertEquals(1, cache2.get(1));
+            assertEquals(1, cache3.get(1));
+        }
+        finally {
+            needStore = false;
+            isClient = false;
+            useExtClassLoader = false;
+        }
+    }
 
-                stopAllGrids();
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testStoreDynamicStart2() throws Exception {
+        try {
+            needStore = false;
+            useExtClassLoader = true;
 
-                g1 = startGrid(1);
-                g2 = startGrid(2);
+            Ignite g1 = startGrid(1);
+            Ignite g2 = startGrid(2);
 
-                isClient = true;
+            isClient = true;
 
-                g3 = startGrid(3);
+            Ignite g3 = startGrid(3);
 
-                isClient = false;
+            Object sf = getExternalClassLoader().loadClass(
+                "org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
 
-                Object sf = ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
+            CacheConfiguration cfg = defaultCacheConfiguration();
 
-                cfg.setCacheStoreFactory((Factory)sf);
-                cfg.setName("customStore");
+            cfg.setCacheStoreFactory((Factory)sf);
+            cfg.setName("customStore");
 
-                cache1 = g1.createCache(cfg);
+            IgniteCache<Integer, Object> cache1 = g1.getOrCreateCache(cfg);
 
-                cache2 = g2.getOrCreateCache(cfg); // Checks deserialization at cache creation.
-                cache3 = g3.getOrCreateCache(cfg); // Checks deserialization at cache creation.
+            IgniteCache<Integer, Object> cache2 = g2.getOrCreateCache("customStore"); // Checks deserialization at cache creation.
+            IgniteCache<Integer, Object> cache3 = g3.getOrCreateCache("customStore"); // Checks deserialization at cache creation.
 
-                cache1.put(1, 1);
+            cache1.put(1, 1);
 
-                assertEquals(1, cache2.get(1));
-                assertEquals(1, cache3.get(1));
-            }
+            assertEquals(1, cache2.get(1));
+            assertEquals(1, cache3.get(1));
         }
         finally {
             needStore = false;
             isClient = false;
-
-            stopAllGrids();
+            useExtClassLoader = false;
         }
     }
 


[08/43] ignite git commit: IGNITE-2329: Implemented a bunch of optimizations: - Garbageless NIO Selector - Get rid of unnecessary ArrayList allocations in GridCacheMvccManager. - Optimized "force keys" futures logic.

Posted by vk...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
index 0a7845b..662cee3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
@@ -231,4 +231,4 @@ public abstract class IgniteTxPreloadAbstractTest extends GridCacheAbstractSelfT
 
         return cfg;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index b4e1ae6..2247acf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -233,8 +234,20 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
 
         List<KeyCacheObject> cacheKeys = F.asList(ctx.toCacheKeyObject(1), ctx.toCacheKeyObject(2));
 
-        ((IgniteKernal)g1).internalCache(null).preloader().request(cacheKeys, new AffinityTopologyVersion(2)).get();
-        ((IgniteKernal)g2).internalCache(null).preloader().request(cacheKeys, new AffinityTopologyVersion(2)).get();
+        IgniteInternalFuture<Object> f1 = ((IgniteKernal)g1).internalCache(null).preloader().request(
+            cacheKeys,
+            new AffinityTopologyVersion(2));
+
+        if (f1 != null)
+            f1.get();
+
+
+        IgniteInternalFuture<Object> f2 = ((IgniteKernal)g2).internalCache(null).preloader().request(
+            cacheKeys,
+            new AffinityTopologyVersion(2));
+
+        if (f2 != null)
+            f2.get();
 
         IgniteCache<Integer, String> cache1 = g1.cache(null);
         IgniteCache<Integer, String> cache2 = g2.cache(null);
@@ -602,4 +615,4 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
             lock1.unlock();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/modules/core/src/test/java/org/apache/ignite/lang/GridTupleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridTupleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridTupleSelfTest.java
index a21ed30..5865cad 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridTupleSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/GridTupleSelfTest.java
@@ -20,10 +20,12 @@ package org.apache.ignite.lang;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.NoSuchElementException;
 import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.lang.GridTupleV;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
 
@@ -102,6 +104,44 @@ public class GridTupleSelfTest extends GridCommonAbstractTest {
     /**
      * JUnit.
      */
+    public void testGridTuple2AsMap() {
+        String str1 = "A test string 1";
+        String str2 = "A test string 2";
+
+        IgniteBiTuple<String, String> tpl = new IgniteBiTuple<>();
+
+        tpl.put(str1, str2);
+
+        assertEquals(str2, tpl.get(str1));
+        assertEquals(1, tpl.size());
+
+        assert tpl.containsKey(str1);
+        assert tpl.containsValue(str2);
+
+        Iterator<Map.Entry<String, String>> it = tpl.entrySet().iterator();
+
+        assert it.hasNext();
+
+        Map.Entry<String, String> next = it.next();
+
+        assertEquals(str1, next.getKey());
+        assertEquals(str2, next.getValue());
+
+        assert !it.hasNext();
+
+        next = F.firstEntry(tpl);
+
+        assertEquals(str1, next.getKey());
+        assertEquals(str2, next.getValue());
+
+        tpl = new IgniteBiTuple<>();
+
+        assert !tpl.entrySet().iterator().hasNext();
+    }
+
+    /**
+     * JUnit.
+     */
     public void testGridTuple3AsIterable() {
         String str1 = "A test string 1";
         String str2 = "A test string 2";
@@ -166,4 +206,4 @@ public class GridTupleSelfTest extends GridCommonAbstractTest {
             info("Caught expected exception: " + e);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/75961eee/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 53eefdd..437a30b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -780,6 +780,7 @@
                                         <exclude>**/NOTICE*</exclude>
                                         <exclude>**/LICENSE*</exclude>
                                         <exclude>src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java</exclude><!--BSD license-->
+                                        <exclude>src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java</exclude><!-- Apache 2.0 license -->
                                         <exclude>src/main/java/org/apache/ignite/internal/util/snaptree/*.java</exclude><!--BSD license-->
                                         <exclude>src/main/java/org/jsr166/*.java</exclude>
                                         <exclude>src/test/java/org/apache/ignite/p2p/p2p.properties</exclude><!--test depends on file content-->


[34/43] ignite git commit: Added test.

Posted by vk...@apache.org.
Added test.


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

Branch: refs/heads/ignite-2249
Commit: 5539cbadb81195c35889b71486b1e449b637d8c0
Parents: 16927ab
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 10 15:32:03 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 10 15:32:03 2016 +0300

----------------------------------------------------------------------
 ...niteCacheEntryListenerExpiredEventsTest.java | 202 +++++++++++++++++++
 1 file changed, 202 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5539cbad/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerExpiredEventsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerExpiredEventsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerExpiredEventsTest.java
new file mode 100644
index 0000000..d9fdfac
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerExpiredEventsTest.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.configuration.CacheEntryListenerConfiguration;
+import javax.cache.configuration.Factory;
+import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryExpiredListener;
+import javax.cache.event.CacheEntryListener;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ModifiedExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteCacheEntryListenerExpiredEventsTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static AtomicInteger evtCntr;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpiredEventAtomic() throws Exception {
+        checkExpiredEvents(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpiredEventAtomicOffheap() throws Exception {
+        checkExpiredEvents(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpiredEventTx() throws Exception {
+        checkExpiredEvents(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpiredEventTxOffheap() throws Exception {
+        checkExpiredEvents(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED));
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void checkExpiredEvents(CacheConfiguration<Object, Object> ccfg) throws Exception {
+        IgniteCache<Object, Object> cache = ignite(0).createCache(ccfg);
+
+        try {
+            evtCntr = new AtomicInteger();
+
+            CacheEntryListenerConfiguration<Object, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+                new ExpiredListenerFactory(),
+                null,
+                true,
+                false
+            );
+
+            cache.registerCacheEntryListener(lsnrCfg);
+
+            IgniteCache<Object, Object> expiryCache =
+                cache.withExpiryPolicy(new ModifiedExpiryPolicy(new Duration(MILLISECONDS, 500)));
+
+            expiryCache.put(1, 1);
+
+            for (int i = 0; i < 10; i++)
+                cache.get(i);
+
+            boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return evtCntr.get() > 0;
+                }
+            }, 5000);
+
+            assertTrue(wait);
+
+            U.sleep(100);
+
+            assertEquals(1, evtCntr.get());
+        }
+        finally {
+            ignite(0).destroyCache(cache.getName());
+        }
+    }
+
+    /**
+     *
+     * @param cacheMode Cache mode.
+     * @param atomicityMode Cache atomicity mode.
+     * @param memoryMode Cache memory mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration(
+        CacheMode cacheMode,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setMemoryMode(memoryMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(1);
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    private static class ExpiredListenerFactory implements Factory<CacheEntryListener<Object, Object>> {
+        /** {@inheritDoc} */
+        @Override public CacheEntryListener<Object, Object> create() {
+            return new ExpiredListener();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ExpiredListener implements CacheEntryExpiredListener<Object, Object> {
+        /** {@inheritDoc} */
+        @Override public void onExpired(Iterable<CacheEntryEvent<?, ?>> evts) {
+            for (CacheEntryEvent<?, ?> evt : evts)
+                evtCntr.incrementAndGet();
+        }
+    }
+}


[19/43] ignite git commit: Renamed the method.

Posted by vk...@apache.org.
Renamed the method.


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

Branch: refs/heads/ignite-2249
Commit: fa246cd2fa8bbf389677f256612124d4a1482afa
Parents: 26719d3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Feb 8 17:55:35 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Feb 8 17:55:35 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fa246cd2/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
index 62420f7..42d6b5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryBasicNameMapper.java
@@ -59,7 +59,7 @@ public class BinaryBasicNameMapper implements BinaryNameMapper {
      *
      * @param isSimpleName Whether to use simple name of class or not.
      */
-    public void setLowerCase(boolean isSimpleName) {
+    public void setSimpleName(boolean isSimpleName) {
         this.isSimpleName = isSimpleName;
     }
 


[13/43] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vk...@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/d844e959
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d844e959
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d844e959

Branch: refs/heads/ignite-2249
Commit: d844e95991408970598b324b204c4b6942e67e13
Parents: f07adff bad0420
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Feb 8 10:31:14 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Feb 8 10:31:14 2016 +0300

----------------------------------------------------------------------
 .../util/nio/SelectedSelectionKeySet.java       | 65 +++++++++++++-------
 1 file changed, 43 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[05/43] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vk...@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/88b0eeb6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/88b0eeb6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/88b0eeb6

Branch: refs/heads/ignite-2249
Commit: 88b0eeb6f81a3c84c3874298d1f2cbb7192bd505
Parents: f3e9ae4 a30f84a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Feb 5 12:24:00 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Feb 5 12:24:00 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAdapter.java       | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[31/43] ignite git commit: IGNITE-2564: CPP: Fixed a bug preventing CPP memory reallocation from Java. This closes #460.

Posted by vk...@apache.org.
IGNITE-2564: CPP: Fixed a bug preventing CPP memory reallocation from Java. This closes #460.


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

Branch: refs/heads/ignite-2249
Commit: c3aa1375171b5f3a97cd50fcd209256e46579b0d
Parents: b7475f0
Author: isapego <is...@gridgain.com>
Authored: Wed Feb 10 15:00:42 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Feb 10 15:00:42 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/core-test/Makefile.am     |  1 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  1 +
 .../project/vs/core-test.vcxproj.filters        |  3 +
 .../platforms/cpp/core-test/src/cache_test.cpp  | 12 +++
 .../cpp/core-test/src/interop_memory_test.cpp   | 95 ++++++++++++++++++++
 .../include/ignite/impl/ignite_environment.h    | 19 ++--
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  2 +-
 .../cpp/core/src/impl/ignite_environment.cpp    | 30 +++++--
 8 files changed, 149 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index aa81c65..531fee0 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -29,6 +29,7 @@ ignite_tests_SOURCES = src/cache_test.cpp \
                          src/cache_query_test.cpp \
                          src/concurrent_test.cpp \
                          src/ignition_test.cpp \
+                         src/interop_memory_test.cpp \
                          src/handle_registry_test.cpp \
                          src/binary_test_defs.cpp \
                          src/binary_reader_writer_raw_test.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 422199a..d98d202 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -40,6 +40,7 @@
     <ClCompile Include="..\..\src\binary_session_test.cpp" />
     <ClCompile Include="..\..\src\binary_test_defs.cpp" />
     <ClCompile Include="..\..\src\cache_query_test.cpp" />
+    <ClCompile Include="..\..\src\interop_memory_test.cpp" />
     <ClCompile Include="..\..\src\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity_messages.cpp" />
   </ItemGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index 32737be..15b9c40 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -34,6 +34,9 @@
     <ClCompile Include="..\..\src\binary_reader_writer_raw_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\interop_memory_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\teamcity_messages.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core-test/src/cache_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_test.cpp b/modules/platforms/cpp/core-test/src/cache_test.cpp
index 32c5bd6..a11865d 100644
--- a/modules/platforms/cpp/core-test/src/cache_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_test.cpp
@@ -476,4 +476,16 @@ BOOST_AUTO_TEST_CASE(TestGetOrCreateCache)
     BOOST_REQUIRE(7 == cache2.Get(5));
 }
 
+BOOST_AUTO_TEST_CASE(TestGetBigString)
+{
+    // Get existing cache
+    cache::Cache<int, std::string> cache = grid0.GetOrCreateCache<int, std::string>("partitioned");
+
+    std::string longStr(impl::IgniteEnvironment::DEFAULT_ALLOCATION_SIZE * 10, 'a');
+
+    cache.Put(5, longStr);
+
+    BOOST_REQUIRE(longStr == cache.Get(5));
+}
+
 BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core-test/src/interop_memory_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/interop_memory_test.cpp b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp
new file mode 100644
index 0000000..07e928c
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+
+using namespace ignite;
+using namespace impl;
+using namespace boost::unit_test;
+
+BOOST_AUTO_TEST_SUITE(MemoryTestSuite)
+
+BOOST_AUTO_TEST_CASE(MemoryReallocationTest)
+{
+    using impl::interop::InteropMemory;
+    using common::concurrent::SharedPointer;
+
+    IgniteEnvironment env;
+
+    SharedPointer<InteropMemory> mem = env.AllocateMemory();
+
+    BOOST_CHECK_EQUAL(mem.Get()->Capacity(), IgniteEnvironment::DEFAULT_ALLOCATION_SIZE);
+
+    BOOST_CHECK(mem.Get()->Data() != NULL);
+
+    // Checking memory for write access.
+    int32_t capBeforeReallocation = mem.Get()->Capacity();
+
+    for (int32_t i = 0; i <capBeforeReallocation; ++i)
+    {
+        int8_t *data = mem.Get()->Data();
+
+        data[i] = static_cast<int8_t>(i);
+    }
+
+    mem.Get()->Reallocate(mem.Get()->Capacity() * 3);
+
+    BOOST_CHECK(mem.Get()->Capacity() >= IgniteEnvironment::DEFAULT_ALLOCATION_SIZE * 3);
+
+    // Checking memory data.
+    for (int32_t i = 0; i <capBeforeReallocation; ++i)
+    {
+        int8_t *data = mem.Get()->Data();
+
+        BOOST_REQUIRE_EQUAL(data[i], static_cast<int8_t>(i));
+    }
+
+    // Checking memory for write access.
+    capBeforeReallocation = mem.Get()->Capacity();
+
+    for (int32_t i = 0; i <capBeforeReallocation; ++i)
+    {
+        int8_t *data = mem.Get()->Data();
+
+        data[i] = static_cast<int8_t>(i + 42);
+    }
+
+    // Trying reallocate memory once more.
+    mem.Get()->Reallocate(mem.Get()->Capacity() * 3);
+
+    // Checking memory data.
+    for (int32_t i = 0; i <capBeforeReallocation; ++i)
+    {
+        int8_t *data = mem.Get()->Data();
+
+        BOOST_REQUIRE_EQUAL(data[i], static_cast<int8_t>(i + 42));
+    }
+
+    BOOST_CHECK(mem.Get()->Capacity() >= IgniteEnvironment::DEFAULT_ALLOCATION_SIZE * 9);
+
+    // Checking memory for write access.
+    memset(mem.Get()->Data(), 0xF0F0F0F0, mem.Get()->Capacity());
+}
+
+BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
index 2fbdb44..02facfc 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -25,16 +25,21 @@
 #include "binary/binary_type_manager.h"
 
 namespace ignite 
-{    
+{
     namespace impl 
     {
         /**
          * Defines environment in which Ignite operates.
          */
         class IGNITE_IMPORT_EXPORT IgniteEnvironment
-        {                
+        {
         public:
             /**
+             * Default memory block allocation size.
+             */
+            enum { DEFAULT_ALLOCATION_SIZE = 1024 };
+
+            /**
              * Default constructor.
              */
             IgniteEnvironment();
@@ -51,7 +56,7 @@ namespace ignite
              * @return JNI handlers.
              */
             ignite::common::java::JniHandlers GetJniHandlers(ignite::common::concurrent::SharedPointer<IgniteEnvironment>* target);
-                
+
             /**
              * Perform initialization on successful start.
              *
@@ -64,8 +69,8 @@ namespace ignite
              *
              * @param memPtr Memory pointer.
              */
-            void OnStartCallback(long long memPtr);        
-            
+            void OnStartCallback(long long memPtr);
+
             /**
              * Get name of Ignite instance.
              *
@@ -120,11 +125,11 @@ namespace ignite
             char* name;
 
             /** Type manager. */
-            binary::BinaryTypeManager* metaMgr;       
+            binary::BinaryTypeManager* metaMgr;
 
             IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment);
         };
-    }    
+    }
 }
 
 #endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
index 08526b5..f66a228 100644
--- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
@@ -379,7 +379,7 @@ namespace ignite
 
                 if (outPtr)
                 {
-                    env.Get()->Context()->TargetInStreamOutStream(javaRef, opType, WriteTo(outMem.Get(), inOp, err), 
+                    env.Get()->Context()->TargetInStreamOutStream(javaRef, opType, outPtr,
                         inMem.Get()->PointerLong(), &jniErr);
 
                     IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3aa1375/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
index 013a139..c9c57a0 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -53,7 +53,23 @@ namespace ignite
             SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
             delete ptr;
-        } 
+        }
+
+        /**
+         * Memory reallocate callback.
+         *
+         * @param target Target environment.
+         * @param memPtr Memory pointer.
+         * @param cap Required capasity.
+         */
+        void IGNITE_CALL MemoryReallocate(void* target, long long memPtr, int cap)
+        {
+            SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+
+            SharedPointer<InteropMemory> mem = env->Get()->GetMemory(memPtr);
+
+            mem.Get()->Reallocate(cap);
+        }
 
         IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(NULL),
             metaMgr(new BinaryTypeManager())
@@ -80,18 +96,20 @@ namespace ignite
             hnds.onStart = OnStart;
             hnds.onStop = OnStop;
 
+            hnds.memRealloc = MemoryReallocate;
+
             hnds.error = NULL;
 
             return hnds;
         }
-            
+
         void IgniteEnvironment::Initialize(SharedPointer<JniContext> ctx)
         {
             this->ctx = ctx;
-                
+
             latch->CountDown();
         }
-        
+
         const char* IgniteEnvironment::InstanceName() const
         {
             return name;
@@ -104,7 +122,7 @@ namespace ignite
 
         SharedPointer<InteropMemory> IgniteEnvironment::AllocateMemory()
         {
-            SharedPointer<InteropMemory> ptr(new InteropUnpooledMemory(1024));
+            SharedPointer<InteropMemory> ptr(new InteropUnpooledMemory(DEFAULT_ALLOCATION_SIZE));
 
             return ptr;
         }
@@ -147,7 +165,7 @@ namespace ignite
             InteropInputStream stream(&mem);
 
             BinaryReaderImpl reader(&stream);
-            
+
             int32_t nameLen = reader.ReadString(NULL, 0);
 
             if (nameLen >= 0)


[20/43] ignite git commit: IGNITE-2509 - Fixed evictions for OFFHEAP_VALUES memory mode - Fixes #452.

Posted by vk...@apache.org.
IGNITE-2509 - Fixed evictions for OFFHEAP_VALUES memory mode - Fixes #452.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-2249
Commit: e2e216d7f1295dd350cc2e98a5b704e27aa92a41
Parents: fa246cd
Author: vershov <ve...@gridgain.com>
Authored: Mon Feb 8 21:54:48 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Feb 8 21:54:48 2016 +0300

----------------------------------------------------------------------
 .../cache/CacheEvictableEntryImpl.java          |   4 +-
 .../GridCacheOffHeapValuesEvictionSelfTest.java | 171 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 3 files changed, 175 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e2e216d7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
index 7cea4f6..9f4d9d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
@@ -103,7 +103,7 @@ public class CacheEvictableEntryImpl<K, V> implements EvictableEntry<K, V> {
 
             byte[] valBytes = null;
 
-            if (cctx.useOffheapEntry())
+            if (cctx.offheapTiered())
                 valBytes = cctx.offheap().get(cctx.swap().spaceName(), cached.partition(), key, keyBytes);
             else {
                 CacheObject cacheObj = cached.valueBytes();
@@ -231,4 +231,4 @@ public class CacheEvictableEntryImpl<K, V> implements EvictableEntry<K, V> {
     @Override public String toString() {
         return S.toString(CacheEvictableEntryImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2e216d7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapValuesEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapValuesEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapValuesEvictionSelfTest.java
new file mode 100644
index 0000000..9baab33
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapValuesEvictionSelfTest.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Test for eviction with offHeap.
+ */
+public class GridCacheOffHeapValuesEvictionSelfTest extends GridCacheAbstractSelfTest {
+
+    private static final int VAL_SIZE = 512 * 1024; // bytes
+    private static final int MAX_VALS_AMOUNT = 100;
+    private static final int MAX_MEMORY_SIZE = MAX_VALS_AMOUNT * VAL_SIZE;
+    private static final int VALS_AMOUNT = MAX_VALS_AMOUNT * 2;
+    private static final int THREAD_COUNT = 4;
+
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutOnHeap() throws Exception {
+        CacheConfiguration<Integer, Object> ccfg = cacheConfiguration(grid(0).name());
+        ccfg.setName("testPutOffHeapValues");
+        ccfg.setStatisticsEnabled(true);
+        ccfg.setOffHeapMaxMemory(MAX_MEMORY_SIZE);
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemorySize(MAX_MEMORY_SIZE);
+
+        ccfg.setSwapEnabled(true);
+        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_VALUES);
+
+        ccfg.setEvictionPolicy(plc);
+
+        final IgniteCache<Integer, Object> cache = grid(0).getOrCreateCache(ccfg);
+
+        fillCache(cache, getTestTimeout());
+
+        assertEquals(VALS_AMOUNT * THREAD_COUNT, cache.size(CachePeekMode.ALL));
+        assertEquals(0, cache.size(CachePeekMode.NEAR));
+        assertEquals(0, cache.size(CachePeekMode.OFFHEAP));
+        assertTrue(MAX_VALS_AMOUNT >= cache.size(CachePeekMode.ONHEAP));
+        assertTrue(MAX_VALS_AMOUNT - 5 <= cache.size(CachePeekMode.ONHEAP));
+        assertEquals(cache.size(CachePeekMode.ALL) - cache.size(CachePeekMode.ONHEAP), cache.size(CachePeekMode.SWAP));
+    }
+
+    /**
+     * swap disabled -> entries discarded
+     * @throws Exception If failed.
+     */
+    public void testPutOnHeapWithOffHeap() throws Exception {
+        final int PLC_MAX_SIZE = 50;
+
+        CacheConfiguration<Integer, Object> ccfg = cacheConfiguration(grid(0).name());
+        ccfg.setName("testPutOnHeapWithOffHeap");
+        ccfg.setStatisticsEnabled(true);
+        ccfg.setOffHeapMaxMemory(MAX_MEMORY_SIZE);
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemorySize(MAX_MEMORY_SIZE);
+        plc.setMaxSize(PLC_MAX_SIZE);
+
+        ccfg.setSwapEnabled(false);
+        ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
+
+        ccfg.setEvictionPolicy(plc);
+
+        final IgniteCache<Integer, Object> cache = grid(0).getOrCreateCache(ccfg);
+
+        fillCache(cache, getTestTimeout());
+
+        assertEquals(cache.size(CachePeekMode.ONHEAP) + cache.size(CachePeekMode.OFFHEAP), cache.size(CachePeekMode.ALL));
+        assertEquals(0, cache.size(CachePeekMode.NEAR));
+        assertEquals(0, cache.size(CachePeekMode.SWAP));
+        assertTrue(PLC_MAX_SIZE >= cache.size(CachePeekMode.ONHEAP));
+        assertTrue(PLC_MAX_SIZE - 5 <= cache.size(CachePeekMode.ONHEAP));
+        assertTrue(MAX_VALS_AMOUNT >= cache.size(CachePeekMode.OFFHEAP));
+        assertTrue(MAX_VALS_AMOUNT - 5 <= cache.size(CachePeekMode.OFFHEAP));
+        assertEquals(cache.size(CachePeekMode.ALL) - cache.size(CachePeekMode.ONHEAP) - cache.size(CachePeekMode.OFFHEAP),
+            cache.size(CachePeekMode.SWAP));
+
+        assertTrue((MAX_VALS_AMOUNT + 5) * VAL_SIZE > cache.metrics().getOffHeapAllocatedSize());
+        assertTrue((MAX_VALS_AMOUNT - 5) * VAL_SIZE < cache.metrics().getOffHeapAllocatedSize());
+    }
+
+    /**
+     * swap enabled -> entries are not discarded
+     * @throws Exception
+     */
+    public void testOnHeapWithOffHeapSwap() throws Exception{
+        final int PLC_MAX_SIZE = 50;
+
+        CacheConfiguration<Integer, Object> ccfg = cacheConfiguration(grid(0).name());
+        ccfg.setName("testOnHeapWithOffHeapSwap");
+        ccfg.setStatisticsEnabled(true);
+        ccfg.setOffHeapMaxMemory(MAX_MEMORY_SIZE);
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemorySize(MAX_MEMORY_SIZE);
+        plc.setMaxSize(PLC_MAX_SIZE);
+
+        ccfg.setSwapEnabled(true);
+        ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
+
+        ccfg.setEvictionPolicy(plc);
+
+        final IgniteCache<Integer, Object> cache = grid(0).getOrCreateCache(ccfg);
+
+        fillCache(cache, getTestTimeout());
+
+        assertEquals(cache.size(CachePeekMode.SWAP) + cache.size(CachePeekMode.ONHEAP) +
+            cache.size(CachePeekMode.OFFHEAP), cache.size(CachePeekMode.ALL));
+
+        assertTrue(PLC_MAX_SIZE >= cache.size(CachePeekMode.ONHEAP));
+        assertTrue(PLC_MAX_SIZE - 5 <= cache.size(CachePeekMode.ONHEAP));
+        assertTrue(MAX_VALS_AMOUNT >= cache.size(CachePeekMode.OFFHEAP));
+        assertTrue(MAX_VALS_AMOUNT - 5 <= cache.size(CachePeekMode.OFFHEAP));
+
+        assertTrue((MAX_VALS_AMOUNT + 5) * VAL_SIZE > cache.metrics().getOffHeapAllocatedSize());
+        assertTrue((MAX_VALS_AMOUNT - 5) * VAL_SIZE < cache.metrics().getOffHeapAllocatedSize());
+    }
+
+    private static void fillCache(final IgniteCache<Integer, Object> cache, long timeout) throws Exception{
+        final byte[] val = new byte[VAL_SIZE];
+        final AtomicInteger keyStart = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(4);
+
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                final int start = keyStart.addAndGet(VALS_AMOUNT);
+
+                for (int i = start; i < start + VALS_AMOUNT; i++)
+                    cache.put(i, val);
+
+                latch.countDown();
+
+                return null;
+            }
+        }, THREAD_COUNT, "test");
+
+        latch.await(timeout, TimeUnit.MILLISECONDS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2e216d7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 88d0834..90125b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredAtomicS
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheOffHeapValuesEvictionSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheP2PUndeploySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCachePartitionedLocalStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCachePartitionedOffHeapLocalStoreSelfTest;
@@ -247,6 +248,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheStorePutxSelfTest.class);
         suite.addTestSuite(GridCacheOffHeapMultiThreadedUpdateSelfTest.class);
         suite.addTestSuite(GridCacheOffHeapAtomicMultiThreadedUpdateSelfTest.class);
+        suite.addTestSuite(GridCacheOffHeapValuesEvictionSelfTest.class);
         suite.addTestSuite(GridCacheColocatedTxStoreExceptionSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedTxStoreExceptionSelfTest.class);
         suite.addTestSuite(GridCacheLocalTxStoreExceptionSelfTest.class);


[26/43] ignite git commit: IGNITE-864 Disabled version check for visorcmd

Posted by vk...@apache.org.
IGNITE-864 Disabled version check for visorcmd


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

Branch: refs/heads/ignite-2249
Commit: c67e2ea5bfb7b73d76744d54047eb1b692e0907a
Parents: b7475f0
Author: Andrey <an...@gridgain.com>
Authored: Wed Feb 10 16:56:23 2016 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Feb 10 16:56:23 2016 +0700

----------------------------------------------------------------------
 .../src/main/scala/org/apache/ignite/visor/visor.scala           | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c67e2ea5/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index a4eed69..335eb9f 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.visor
 
+import org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER
 import org.apache.ignite._
 import org.apache.ignite.cluster.{ClusterGroup, ClusterGroupEmptyException, ClusterMetrics, ClusterNode}
 import org.apache.ignite.events.EventType._
@@ -303,6 +304,9 @@ object visor extends VisorTag {
         }
     })
 
+    // Make sure visor starts without version checker print.
+    System.setProperty(IGNITE_UPDATE_NOTIFIER, "false")
+
     addHelp(
         name = "mlist",
         shortInfo = "Prints Visor console memory variables.",


[39/43] ignite git commit: added missing file header to org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactorySelfTest

Posted by vk...@apache.org.
added missing file header to org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactorySelfTest

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-2249
Commit: 0491a5f814365723b53ff1820aad76f646307f13
Parents: a9937a6
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Feb 10 18:54:36 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Feb 10 19:31:41 2016 +0300

----------------------------------------------------------------------
 .../KerberosHadoopFileSystemFactorySelfTest.java   | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0491a5f8/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
index 8fb1612..ea7fa99 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.ignite.hadoop.fs;
 
 import java.io.ByteArrayInputStream;


[21/43] ignite git commit: IGNITE-1563 .NET: Implemented "atomics": AtomicReference and AtomicSequence. This closes #455.

Posted by vk...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/f7c1296c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index ad62f38..cdcddf0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -281,6 +281,40 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             }
         }
 
+        internal static IUnmanagedTarget ProcessorAtomicSequence(IUnmanagedTarget target, string name, long initialValue, 
+            bool create)
+        {
+            var name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                var res = JNI.ProcessorAtomicSequence(target.Context, target.Target, name0, initialValue, create);
+
+                return res == null ? null : target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
+        internal static IUnmanagedTarget ProcessorAtomicReference(IUnmanagedTarget target, string name, long memPtr, 
+            bool create)
+        {
+            var name0 = IgniteUtils.StringToUtf8Unmanaged(name);
+
+            try
+            {
+                var res = JNI.ProcessorAtomicReference(target.Context, target.Target, name0, memPtr, create);
+
+                return res == null ? null : target.ChangeTarget(res);
+            }
+            finally
+            {
+                Marshal.FreeHGlobal(new IntPtr(name0));
+            }
+        }
+
         internal static void ProcessorGetIgniteConfiguration(IUnmanagedTarget target, long memPtr)
         {
             JNI.ProcessorGetIgniteConfiguration(target.Context, target.Target, memPtr);
@@ -867,6 +901,51 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             JNI.AtomicLongClose(target.Context, target.Target);
         }
 
+        internal static long AtomicSequenceGet(IUnmanagedTarget target)
+        {
+            return JNI.AtomicSequenceGet(target.Context, target.Target);
+        }
+
+        internal static long AtomicSequenceIncrementAndGet(IUnmanagedTarget target)
+        {
+            return JNI.AtomicSequenceIncrementAndGet(target.Context, target.Target);
+        }
+
+        internal static long AtomicSequenceAddAndGet(IUnmanagedTarget target, long value)
+        {
+            return JNI.AtomicSequenceAddAndGet(target.Context, target.Target, value);
+        }
+
+        internal static int AtomicSequenceGetBatchSize(IUnmanagedTarget target)
+        {
+            return JNI.AtomicSequenceGetBatchSize(target.Context, target.Target);
+        }
+
+        internal static void AtomicSequenceSetBatchSize(IUnmanagedTarget target, int size)
+        {
+            JNI.AtomicSequenceSetBatchSize(target.Context, target.Target, size);
+        }
+
+        internal static bool AtomicSequenceIsClosed(IUnmanagedTarget target)
+        {
+            return JNI.AtomicSequenceIsClosed(target.Context, target.Target);
+        }
+
+        internal static void AtomicSequenceClose(IUnmanagedTarget target)
+        {
+            JNI.AtomicSequenceClose(target.Context, target.Target);
+        }
+
+        internal static bool AtomicReferenceIsClosed(IUnmanagedTarget target)
+        {
+            return JNI.AtomicReferenceIsClosed(target.Context, target.Target);
+        }
+
+        internal static void AtomicReferenceClose(IUnmanagedTarget target)
+        {
+            JNI.AtomicReferenceClose(target.Context, target.Target);
+        }
+
         internal static bool ListenableCancel(IUnmanagedTarget target)
         {
             return JNI.ListenableCancel(target.Context, target.Target);